steveloughran commented on code in PR #1244: URL: https://github.com/apache/parquet-java/pull/1244#discussion_r1439075950
########## parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/DelegatePathOutputCommitter.java: ########## @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.parquet.hadoop.util; + +import java.io.IOException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; + +/** + * Proxy pattern for class <code>PathOutputCommitter</code> + */ +public class DelegatePathOutputCommitter extends PathOutputCommitter { Review Comment: this is already in org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter also look at org.apache.hadoop.mapreduce.lib.output.BindingPathOutputCommitter ########## parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputCommitter.java: ########## @@ -20,30 +20,46 @@ import java.io.IOException; import java.util.List; +import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory; import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel; import org.apache.parquet.hadoop.util.ContextUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * decorator class for PathOutputCommitter, for adding optional parquet metadata file, + * overriding all methods to delegate to an instance of <code>PathOutputCommitterFactory.getCommitterFactory(..)</code> + * <p> + * For internal reason (hard-coded 'instanceof FileOutputCommitter' in spark source code), + * this class still temporarily <code>extends FileOutputCommitter</code> instead of <code>PathOutputCommitter</code> + */ public class ParquetOutputCommitter extends FileOutputCommitter { private static final Logger LOG = LoggerFactory.getLogger(ParquetOutputCommitter.class); private final Path outputPath; + private final FileOutputCommitter delegate; + public ParquetOutputCommitter(Path outputPath, TaskAttemptContext context) throws IOException { super(outputPath, context); this.outputPath = outputPath; + this.delegate = (FileOutputCommitter) + PathOutputCommitterFactory.getCommitterFactory(outputPath, context.getConfiguration()) Review Comment: this will break with s3a or manifest committers which aren't subclasses of FOC -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
