You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/12/02 14:45:00 UTC

[GitHub] [iceberg] pvary opened a new pull request #1861: Hive: OutputCommitter implementation for Hive writes

pvary opened a new pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861


   Separated out the HiveIcebergOutputCommitter related stuff from #1407 so it is easier to review. HiveIcebergRecordWriter was also needed since the 2 are tightly coupled.
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536913543



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Record record = ((Container<Record>) row).get();
+
+    // Update the current key with the record, so we do not create a new object for every record
+    currentKey.partition(record);
+
+    AppenderWrapper currentAppender = openAppenders.get(currentKey);
+    if (currentAppender == null) {
+      currentAppender = getAppender();
+      openAppenders.put(currentKey.copy(), currentAppender);
+    }
+
+    currentAppender.appender.add(record);
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    // Close the open appenders and store the closed file data
+    for (PartitionKey key : openAppenders.keySet()) {
+      AppenderWrapper wrapper = openAppenders.get(key);
+      wrapper.close();
+      closedFileData.put(key,
+          new ClosedFileData(key, wrapper.location, fileFormat, wrapper.length(), wrapper.metrics()));
+    }
+
+    openAppenders.clear();
+
+    // If abort then remove the unnecessary files
+    if (abort) {
+      Tasks.foreach(closedFileData.values().stream().map(ClosedFileData::fileName).iterator())
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exception) -> LOG.debug("Failed on to remove file {} on abort", file, exception))
+          .run(io::deleteFile);
+    }
+    LOG.info("IcebergRecordWriter is closed. Created {} files", closedFileData.values());
+  }
+
+  @Override
+  public void close(org.apache.hadoop.mapreduce.TaskAttemptContext context) throws IOException {
+    close(false);
+  }
+
+  @Override
+  public void close(Reporter reporter) throws IOException {
+    close(false);
+  }
+
+  public Set<ClosedFileData> closedFileData() {
+    return new HashSet<>(closedFileData.values());
+  }
+
+  static HiveIcebergRecordWriter removeWriter(TaskAttemptID taskAttemptID) {
+    return writers.remove(taskAttemptID);
+  }

Review comment:
       Can you move this static method up to the top near the static `writers` map? Seems like that should all be kept together.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537818285



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.io.PartitionedFanoutWriter;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends PartitionedFanoutWriter<Record>

Review comment:
       How much is per writer?
   
   My approach is: I would like to see the writes working. We can optimize later.
   
   What I see as a problem here is that the data should be sorted, or at least grouped by partition key, which I think is a non-trivial task without changing the query.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535462545



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/ClosedFileData.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.Serializable;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * Class for storing the data file properties which are needed for an Icebreg commit.

Review comment:
       Fixed




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536910700



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));

Review comment:
       Is it possible to serialize the table's `FileIO` to tasks instead of using a new `HadoopFileIO` here?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537796544



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
##########
@@ -133,4 +146,93 @@ public DecomposedPredicate decomposePredicate(JobConf jobConf, Deserializer dese
     predicate.pushedPredicate = (ExprNodeGenericFuncDesc) exprNodeDesc;
     return predicate;
   }
+
+  /**
+   * Returns the Table FileIO serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table FileIO object
+   */
+  public static FileIO io(Configuration configuration) {
+    return (FileIO) get(configuration, InputFormatConfig.FILE_IO);
+  }
+
+  /**
+   * Returns the Table LocationProvider serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table LocationProvider object
+   */
+  public static LocationProvider location(Configuration configuration) {
+    return (LocationProvider) get(configuration, InputFormatConfig.LOCATION_PROVIDER);
+  }
+
+  /**
+   * Returns the Table EncryptionManager serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table EncryptionManager object
+   */
+  public static EncryptionManager encryption(Configuration configuration) {
+    return (EncryptionManager) get(configuration, InputFormatConfig.ENCRYPTION_MANAGER);
+  }
+
+  /**
+   * Returns the Table Schema serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table Schema object
+   */
+  public static Schema schema(Configuration configuration) {
+    return SchemaParser.fromJson(configuration.get(InputFormatConfig.TABLE_SCHEMA));
+  }
+
+  /**
+   * Returns the Table PartitionSpec serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table PartitionSpec object
+   */
+  public static PartitionSpec spec(Configuration configuration) {
+    return PartitionSpecParser.fromJson(schema(configuration), configuration.get(InputFormatConfig.PARTITION_SPEC));
+  }
+
+  /**
+   * Stores the serializable table data in the configuration.
+   * Currently the following is handled:
+   * <ul>
+   *   <li>- Location</li>
+   *   <li>- Schema</li>
+   *   <li>- Partition specification</li>
+   *   <li>- FileIO for handling table files</li>
+   *   <li>- Location provider used for file generation</li>
+   *   <li>- Encryption manager for encryption handling</li>
+   * </ul>
+   * @param configuration The target configuration to store to
+   * @param table The table which we want to store to the configuration
+   */
+  @VisibleForTesting
+  static void put(Configuration configuration, Table table) {
+    configuration.set(InputFormatConfig.TABLE_LOCATION, table.location());
+    configuration.set(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(table.schema()));
+    configuration.set(InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(table.spec()));
+
+    put(configuration, InputFormatConfig.FILE_IO, table.io());
+    put(configuration, InputFormatConfig.LOCATION_PROVIDER, table.locationProvider());
+    put(configuration, InputFormatConfig.ENCRYPTION_MANAGER, table.encryption());
+  }
+
+  private static void put(Configuration configuration, String key, Serializable object) {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         ObjectOutputStream oos = new ObjectOutputStream(baos)) {
+      oos.writeObject(object);
+      configuration.set(key, Base64.getEncoder().encodeToString(baos.toByteArray()));
+    } catch (IOException ioe) {
+      throw new RuntimeException(String.format("Error serializing %s to configuration", object), ioe);
+    }
+  }
+
+  private static Object get(Configuration configuration, String key) {

Review comment:
       Could this class delegate to `SerializationUtil.serializeToBase64` and `SerializationUtil.deserializeFromBase64`?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537549470



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);
+      } else {
+        LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job.
+    // Intentionally used foreach on a single item. Using the Tasks API here only for the retry capability.
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true /* recursive */);

Review comment:
       If there are runaway tasks they might write into the temp directory even after the job is finished.
   So I feel that it is ok to clean this as a best effort. Do you agree?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537794965



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.io.PartitionedFanoutWriter;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends PartitionedFanoutWriter<Record>

Review comment:
       We generally don't recommend using the fanout writer because it takes so much memory and could easily produce a ton of tiny files. Are you sure you want to use it instead of the normal partitioned writer?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535464193



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",
+            System.currentTimeMillis() - startTime, table, addedFiles);
+      } else {
+        LOG.info("Commit for Iceberg write taken {} ms for {} with no new files",
+            System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true /* recursive */);
+          } catch (IOException e) {
+            throw new RuntimeIOException(e, "Failed to delete job directory: %s", file);
+          }
+        });
+    cleanupJob(jobContext);
+  }
+
+  private static void createToCommitFile(Set<ClosedFileData> closedFiles, String location, FileIO io)

Review comment:
       That was a long debate between me and myself 😄
   If you say `FileForCommit` is better, then I am happy to change it.
   
   Try to change all occurrences :)
   

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",

Review comment:
       Done.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537794965



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.io.PartitionedFanoutWriter;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends PartitionedFanoutWriter<Record>

Review comment:
       We generally don't recommend using the fanout writer because it takes so much memory. Are you sure you want to use it instead of the normal partitioned writer?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537847413



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveOutputFileFactory.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.LocationProvider;
+import org.apache.iceberg.io.OutputFileFactory;
+
+public class HiveOutputFileFactory extends OutputFileFactory {
+  private final TaskAttemptID taskAttemptID;
+
+  public HiveOutputFileFactory(PartitionSpec spec, FileFormat fileFormat, LocationProvider locationProvider, FileIO io,
+                               EncryptionManager encryptionManager, TaskAttemptID taskAttemptID) {
+    super(spec, fileFormat, locationProvider, io, encryptionManager, 0, 0);
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  /**
+   * Override the filename generation so it contains jobId, taskId, taskAttemptId. Kept the UUID and the fileCount so
+   * the filenames are similar for other writers.
+   * @return The generated file name
+   */
+  @Override
+  protected String generateFilename() {
+    return format().addExtension(
+        String.format("%05d-%d-%d-%s-%05d", taskAttemptID.getJobID().getId(), taskAttemptID.getTaskID().getId(),

Review comment:
       The UUID was supposed to be a UUID for the write, not for each task. Looks like that was broken at some point by refactoring.
   
   In that case, What do you think about adding a constructor that accepts a UUID string? Then you could call it with an identifier that works for your job, whether that's a job-level UUID from config or a Hadoop Job ID. And we can follow up in the other engines and generate the UUID correctly.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535463141



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/ClosedFileData.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.Serializable;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * Class for storing the data file properties which are needed for an Icebreg commit.
+ * <ul>
+ *   <li>Partition key
+ *   <li>File name
+ *   <li>File format
+ *   <li>File size
+ *   <li>Metrics
+ * </ul>
+ */
+final class ClosedFileData implements Serializable {
+  private final PartitionKey partitionKey;
+  private final String fileName;
+  private final FileFormat fileFormat;
+  private final Long length;
+  private final Metrics metrics;
+
+  ClosedFileData(PartitionKey partitionKey, String fileName, FileFormat fileFormat, Long length, Metrics metrics) {
+    this.partitionKey = partitionKey;
+    this.fileName = fileName;
+    this.fileFormat = fileFormat;
+    this.length = length;
+    this.metrics = metrics;
+  }
+
+  PartitionKey partitionKey() {
+    return partitionKey;
+  }
+
+  String fileName() {
+    return fileName;
+  }
+
+  FileFormat fileFormat() {
+    return fileFormat;
+  }
+
+  Long length() {

Review comment:
       Done. For the record the name was inherited from the appender length(), but fileSize is better.
   Renamed




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535470281



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/LocationHelper.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+class LocationHelper {
+  private static final String TO_COMMIT_EXTENSION = ".toCommit";
+
+  private LocationHelper() {
+  }
+
+  /**
+   * Generates query directory location based on the configuration.
+   * Currently it uses tableLocation/queryId
+   * @param conf The job's configuration
+   * @return The directory to store the query result files
+   */
+  static String generateQueryLocation(Configuration conf) {
+    String tableLocation = conf.get(InputFormatConfig.TABLE_LOCATION);
+    String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    return tableLocation + "/" + queryId;
+  }
+
+  /**
+   * Generates the job temp location based on the job configuration.
+   * Currently it uses QUERY_LOCATION/jobId.
+   * @param conf The job's configuration
+   * @param jobId The JobID for the task
+   * @return The file to store the results
+   */
+  static String generateJobLocation(Configuration conf, JobID jobId) {
+    return generateQueryLocation(conf) + "/" + jobId;
+  }
+
+  /**
+   * Generates datafile location based on the task configuration.
+   * Currently it uses QUERY_LOCATION/jobId/taskAttemptId.
+   * @param conf The job's configuration
+   * @param taskAttemptId The TaskAttemptID for the task
+   * @return The file to store the results
+   */
+  static String generateDataFileLocation(Configuration conf, TaskAttemptID taskAttemptId) {
+    return generateJobLocation(conf, taskAttemptId.getJobID()) + "/" + taskAttemptId.toString();
+  }
+
+  /**
+   * Generates file location based on the task configuration and a specific task id.
+   * This file will be used to store the data required to generate the Iceberg commit.
+   * Currently it uses QUERY_LOCATION/jobId/task-[0..numTasks].toCommit.

Review comment:
       I did not meant to be so mathematically correct, but you are absolutely right 😄 




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536910273



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {

Review comment:
       You might consider moving the builder call into the `ClosedFileData` class as `toDataFile`, if you don't simply serialize `DataFile` instead of `ClosedFileData`.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537544771



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;

Review comment:
       Added LocationProvider serialization




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537537553



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/ClosedFileData.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.Serializable;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * Class for storing the data file properties which are needed for an Iceberg commit.
+ * <ul>
+ *   <li>Partition key
+ *   <li>File name
+ *   <li>File format
+ *   <li>File size
+ *   <li>Metrics
+ * </ul>
+ */
+final class ClosedFileData implements Serializable {

Review comment:
       Serializable DataFile[] is enough.
   Thanks for pointing out!

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();

Review comment:
       Changed

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();

Review comment:
       Changed




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-737274387


   @marton-bod: Could you please review?
   Thanks,
   Peter


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537544438



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>

Review comment:
       I prefer to have it in the name of the classes. When we have to debug code in Hive it could be very helpful to see that we are looking at an Iceberg class without examining the package first.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-740193832


   @pvary, it looks like this now includes commits you didn't intend to add. Can you take a look at your branch? I think it's about ready to merge (we can fix the remaining issues later).


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536912564



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;

Review comment:
       Ideally, `LocationProvider` and `FileIO` should be passed in and should come from the table implementation. Those are required to be `Serializable`.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537540337



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));

Review comment:
       Serialized FileIO/LocationProvider/EncryptionManager




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536911852



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/LocationHelper.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+class LocationHelper {
+  private static final String FOR_COMMIT_EXTENSION = ".forCommit";
+
+  private LocationHelper() {
+  }
+
+  /**
+   * Generates query directory location based on the configuration.
+   * Currently it uses tableLocation/queryId
+   * @param conf The job's configuration
+   * @return The directory to store the query result files
+   */
+  static String generateQueryLocation(Configuration conf) {
+    String tableLocation = conf.get(InputFormatConfig.TABLE_LOCATION);
+    String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    return tableLocation + "/" + queryId;

Review comment:
       Can we add a prefix between table location and query ID, like "temp"?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/LocationHelper.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+class LocationHelper {
+  private static final String FOR_COMMIT_EXTENSION = ".forCommit";
+
+  private LocationHelper() {
+  }
+
+  /**
+   * Generates query directory location based on the configuration.
+   * Currently it uses tableLocation/queryId
+   * @param conf The job's configuration
+   * @return The directory to store the query result files
+   */
+  static String generateQueryLocation(Configuration conf) {
+    String tableLocation = conf.get(InputFormatConfig.TABLE_LOCATION);
+    String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    return tableLocation + "/" + queryId;

Review comment:
       Can we add a level between table location and query ID, like "temp"?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536911754



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);
+      } else {
+        LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {

Review comment:
       I think the implementation of this is actual correct for `cleanupJob`, which is not implemented.
   
   Both `commitJob` and `abortJob` should clean up the committer's data files, which are the files created by each task in `commitTask` that hold the `DataFile` list. Both functions should call `cleanupJob` to do that, and that's what this function appears to do.
   
   Before `abortJob` calls `cleanupJob`, I think it should delete all of the data files that were written but will not be committed. So I think this function should actually call `readFileForCommit` via `Tasks` to generate a list of data files, then call `Tasks` again to delete each data file (that way both operations are independently retried).




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536908568



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();

Review comment:
       In Spark, we just use `DataFile[]`.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535465228



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)

Review comment:
       I do not see too much point adding a new configuration for this.
   If we see some problems we can introduce the config. Especially now, that S3 supposed to have consistent listing.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,

Review comment:
       Done




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537542316



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();

Review comment:
       I prefer to do it in another PR. I do not see how specific writes could be handled with SerDe-s. This might be trivial, but since I have too many things in progress I prefer to not to delve into a new issue before closing up some already open tasks.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536912119



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>

Review comment:
       Minor: Since this lives in Iceberg, we can probably remove Iceberg from the class names.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535470740



##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.junit.Assert;
+
+public class HiveIcebergTestUtils {

Review comment:
       I think it would be a good idea to have them merged.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r540402654



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,291 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final String FOR_COMMIT_EXTENSION = ".forCommit";
+
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  /**
+   * Collects the generated data files and creates a commit file storing the data file list.
+   * @param context The job context
+   * @throws IOException Thrown if there is an error writing the commit file
+   */
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = generateFileForCommitLocation(context.getJobConf(),
+        attemptID.getJobID(), attemptID.getTaskID().getId());
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    DataFile[] closedFiles;
+    if (writer != null) {
+      closedFiles = writer.dataFiles();
+    } else {
+      closedFiles = new DataFile[0];
+    }
+
+    // Creating the file containing the data files generated by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, HiveIcebergStorageHandler.io(context.getJobConf()));
+  }
+
+  /**
+   * Removes files generated by this task.
+   * @param context The task context
+   * @throws IOException Thrown if there is an error closing the writer
+   */
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  /**
+   * Reads the commit files stored in the temp directory and collects the generated committed data files.
+   * Appends the data files to the table. At the end removes the temporary directory.
+   * @param jobContext The job context
+   * @throws IOException if there is a failure deleting the files
+   */
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}", table,
+        generateJobLocation(conf, jobContext.getJobID()));
+
+    FileIO io = HiveIcebergStorageHandler.io(jobContext.getJobConf());
+    List<DataFile> dataFiles = dataFiles(jobContext, io, true);
+
+    if (dataFiles.size() > 0) {
+      // Appending data files to the table
+      AppendFiles append = table.newAppend();
+      dataFiles.forEach(append::appendFile);
+      append.commit();
+      LOG.info("Commit took {} ms for table: {} with {} file(s)", System.currentTimeMillis() - startTime, table,
+          dataFiles.size());
+      LOG.debug("Added files {}", dataFiles);
+    } else {
+      LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Removes the generated data files, if there is a commit file already generated for them.
+   * The cleanup at the end removes the temporary directory as well.
+   * @param jobContext The job context
+   * @param status The status of the job
+   * @throws IOException if there is a failure deleting the files
+   */
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    FileIO io = HiveIcebergStorageHandler.io(jobContext.getJobConf());
+    List<DataFile> dataFiles = dataFiles(jobContext, io, false);
+
+    // Check if we have files already committed and remove data files if there are any
+    if (dataFiles.size() > 0) {
+      Tasks.foreach(dataFiles)
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exc) -> LOG.debug("Failed on to remove data file {} on abort job", file.path(), exc))
+          .run(file -> io.deleteFile(file.path().toString()));
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Cleans up the jobs temporary location.
+   * @param jobContext The job context
+   * @throws IOException if there is a failure deleting the files
+   */
+  private void cleanup(JobContext jobContext) throws IOException {
+    String location = generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Cleaning for job: {} on location: {}", jobContext.getJobID(), location);
+
+    // Remove the job's temp directory recursively.
+    // Intentionally used foreach on a single item. Using the Tasks API here only for the retry capability.
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on cleanup job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());

Review comment:
       Nit: I would prefer if cleanup happened by removing the expected task commit files one-by-one rather than deleting a directory because it could use `FileIO`. I understand that this is intended to drop the folder as well for stores that track folders. Maybe a follow-up to add a `deletePrefix` to `FileIO` would fix it.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] marton-bod commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
marton-bod commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535310601



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/ClosedFileData.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.Serializable;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * Class for storing the data file properties which are needed for an Icebreg commit.

Review comment:
       nit: typo in word Iceberg

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/ClosedFileData.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.Serializable;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * Class for storing the data file properties which are needed for an Icebreg commit.
+ * <ul>
+ *   <li>Partition key
+ *   <li>File name
+ *   <li>File format
+ *   <li>File size
+ *   <li>Metrics
+ * </ul>
+ */
+final class ClosedFileData implements Serializable {
+  private final PartitionKey partitionKey;
+  private final String fileName;
+  private final FileFormat fileFormat;
+  private final Long length;
+  private final Metrics metrics;
+
+  ClosedFileData(PartitionKey partitionKey, String fileName, FileFormat fileFormat, Long length, Metrics metrics) {
+    this.partitionKey = partitionKey;
+    this.fileName = fileName;
+    this.fileFormat = fileFormat;
+    this.length = length;
+    this.metrics = metrics;
+  }
+
+  PartitionKey partitionKey() {
+    return partitionKey;
+  }
+
+  String fileName() {
+    return fileName;
+  }
+
+  FileFormat fileFormat() {
+    return fileFormat;
+  }
+
+  Long length() {

Review comment:
       can we rename this to `fileSize()` to align with the javadoc and the above two methods? Also maybe a short javadoc comment on what unit we use here (I'm assuming bytes)

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",
+            System.currentTimeMillis() - startTime, table, addedFiles);
+      } else {
+        LOG.info("Commit for Iceberg write taken {} ms for {} with no new files",
+            System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job
+    Tasks.foreach(location)

Review comment:
       just to clarify: `Tasks` is only used here for the retry feature? location seems to be a single string, so I guess there'd be no parallel execution here

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Preconditions.checkArgument(row instanceof Container);

Review comment:
       can you add a short comment on why this needs to be a Container instance?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",
+            System.currentTimeMillis() - startTime, table, addedFiles);
+      } else {
+        LOG.info("Commit for Iceberg write taken {} ms for {} with no new files",
+            System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true /* recursive */);
+          } catch (IOException e) {
+            throw new RuntimeIOException(e, "Failed to delete job directory: %s", file);
+          }
+        });
+    cleanupJob(jobContext);
+  }
+
+  private static void createToCommitFile(Set<ClosedFileData> closedFiles, String location, FileIO io)
+      throws IOException {
+
+    OutputFile commitFile = io.newOutputFile(location);
+    ObjectOutputStream oos = new ObjectOutputStream(commitFile.createOrOverwrite());

Review comment:
       can we use try-with-resources here to make sure the stream is closed?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",

Review comment:
       nit phrasing: "Commit took {} ms for table: {} with file(s): {}"

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,

Review comment:
       nit: it might help to include labels for the placeholders
   e.g. "Committing job has started for table: {}, using location: {}, expecting {} file(s)."

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)

Review comment:
       do we want to make the retry count configurable?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",
+            System.currentTimeMillis() - startTime, table, addedFiles);
+      } else {
+        LOG.info("Commit for Iceberg write taken {} ms for {} with no new files",
+            System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true /* recursive */);
+          } catch (IOException e) {
+            throw new RuntimeIOException(e, "Failed to delete job directory: %s", file);
+          }
+        });
+    cleanupJob(jobContext);
+  }
+
+  private static void createToCommitFile(Set<ClosedFileData> closedFiles, String location, FileIO io)

Review comment:
       I know this is named `ToCommit` because of the file extension introduced in the Writer. But I personally find this wording a little confusing here (especially before scrolling down to the Writer code) - what do you think about naming this something like `createFileForCommit`? similarly for the read

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Preconditions.checkArgument(row instanceof Container);
+
+    Record record = ((Container<Record>) row).get();
+
+    currentKey.partition(record);
+
+    AppenderWrapper currentAppender = openAppenders.get(currentKey);
+    if (currentAppender == null) {
+      currentAppender = getAppender();
+      openAppenders.put(currentKey.copy(), currentAppender);
+    }
+
+    currentAppender.appender.add(record);
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    // Close the open appenders and store the closed file data
+    for (PartitionKey key : openAppenders.keySet()) {
+      AppenderWrapper wrapper = openAppenders.get(key);
+      wrapper.close();
+      closedFileData.put(key,
+          new ClosedFileData(key, wrapper.location, fileFormat, wrapper.length(), wrapper.metrics()));
+    }
+
+    openAppenders.clear();
+
+    // If abort then remove the unnecessary files
+    if (abort) {
+      Tasks.foreach(closedFileData.values().stream().map(ClosedFileData::fileName).iterator())
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exception) -> LOG.debug("Failed on to remove file {} on abort", file, exception))
+          .run(io::deleteFile);
+    }
+    LOG.info("IcebergRecordWriter is closed. Created {} files", closedFileData);

Review comment:
       did you mean closedFileData.size() or you wanted to list the map contents?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/LocationHelper.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+class LocationHelper {
+  private static final String TO_COMMIT_EXTENSION = ".toCommit";
+
+  private LocationHelper() {
+  }
+
+  /**
+   * Generates query directory location based on the configuration.
+   * Currently it uses tableLocation/queryId
+   * @param conf The job's configuration
+   * @return The directory to store the query result files
+   */
+  static String generateQueryLocation(Configuration conf) {
+    String tableLocation = conf.get(InputFormatConfig.TABLE_LOCATION);
+    String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    return tableLocation + "/" + queryId;
+  }
+
+  /**
+   * Generates the job temp location based on the job configuration.
+   * Currently it uses QUERY_LOCATION/jobId.
+   * @param conf The job's configuration
+   * @param jobId The JobID for the task
+   * @return The file to store the results
+   */
+  static String generateJobLocation(Configuration conf, JobID jobId) {
+    return generateQueryLocation(conf) + "/" + jobId;
+  }
+
+  /**
+   * Generates datafile location based on the task configuration.
+   * Currently it uses QUERY_LOCATION/jobId/taskAttemptId.
+   * @param conf The job's configuration
+   * @param taskAttemptId The TaskAttemptID for the task
+   * @return The file to store the results
+   */
+  static String generateDataFileLocation(Configuration conf, TaskAttemptID taskAttemptId) {
+    return generateJobLocation(conf, taskAttemptId.getJobID()) + "/" + taskAttemptId.toString();
+  }
+
+  /**
+   * Generates file location based on the task configuration and a specific task id.
+   * This file will be used to store the data required to generate the Iceberg commit.
+   * Currently it uses QUERY_LOCATION/jobId/task-[0..numTasks].toCommit.

Review comment:
       shouldn't this be `task-[0..numTasks).toCommit`? (open interval on the right)

##########
File path: mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.File;
+import java.io.IOException;
+import java.time.OffsetDateTime;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.IcebergGenerics;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.junit.Assert;
+
+public class HiveIcebergTestUtils {

Review comment:
       Just to check: will this be merged with the same named new class from https://github.com/apache/iceberg/pull/1854?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536913293



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);
+      } else {
+        LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job.
+    // Intentionally used foreach on a single item. Using the Tasks API here only for the retry capability.
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true /* recursive */);

Review comment:
       This won't fail if the path doesn't exist, but if you delete just the file directly, you'd want to check that it exists first, since some tasks may not have finished and committed and have aborted instead.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537539006



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)

Review comment:
       Added throwFailureWhenFinished, but did not used stopOnFailure. This way I was able to reuse the code for abort and for commit. Since this is only for the exception handing I think this could be an acceptable compromise

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/LocationHelper.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+class LocationHelper {
+  private static final String FOR_COMMIT_EXTENSION = ".forCommit";
+
+  private LocationHelper() {
+  }
+
+  /**
+   * Generates query directory location based on the configuration.
+   * Currently it uses tableLocation/queryId
+   * @param conf The job's configuration
+   * @return The directory to store the query result files
+   */
+  static String generateQueryLocation(Configuration conf) {
+    String tableLocation = conf.get(InputFormatConfig.TABLE_LOCATION);
+    String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    return tableLocation + "/" + queryId;
+  }
+
+  /**
+   * Generates the job temp location based on the job configuration.
+   * Currently it uses QUERY_LOCATION/jobId.
+   * @param conf The job's configuration
+   * @param jobId The JobID for the task
+   * @return The file to store the results
+   */
+  static String generateJobLocation(Configuration conf, JobID jobId) {
+    return generateQueryLocation(conf) + "/" + jobId;
+  }
+
+  /**
+   * Generates datafile location based on the task configuration.
+   * Currently it uses QUERY_LOCATION/jobId/taskAttemptId.
+   * @param conf The job's configuration
+   * @param taskAttemptId The TaskAttemptID for the task
+   * @return The file to store the results
+   */
+  static String generateDataFileLocation(Configuration conf, TaskAttemptID taskAttemptId) {
+    return generateJobLocation(conf, taskAttemptId.getJobID()) + "/" + taskAttemptId.toString();
+  }
+
+  /**
+   * Generates file location based on the task configuration and a specific task id.
+   * This file will be used to store the data required to generate the Iceberg commit.
+   * Currently it uses QUERY_LOCATION/jobId/task-[0..numTasks).forCommit.
+   * @param conf The job's configuration
+   * @param jobId The jobId for the task
+   * @param taskId The taskId for the commit file
+   * @return The file to store the results
+   */
+  static String generateFileForCommitLocation(Configuration conf, JobID jobId, int taskId) {
+    return generateJobLocation(conf, jobId) + "/task-" + taskId + FOR_COMMIT_EXTENSION;
+  }
+
+  /**
+   * Generates file location location based on the task configuration.
+   * This file will be used to store the data required to generate the Iceberg commit.
+   * Currently it uses QUERY_LOCATION/jobId/task-[0..numTasks).forCommit.
+   * @param conf The job's configuration
+   * @param taskAttemptId The TaskAttemptID for the task
+   * @return The file to store the results
+   */
+  static String generateFileForCommitLocation(Configuration conf, TaskAttemptID taskAttemptId) {

Review comment:
       Removed

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {

Review comment:
       Removed ClosedFileData and started using DataFile[]

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);

Review comment:
       Added specific debug log for the added files




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537824333



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveOutputFileFactory.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.LocationProvider;
+import org.apache.iceberg.io.OutputFileFactory;
+
+public class HiveOutputFileFactory extends OutputFileFactory {
+  private final TaskAttemptID taskAttemptID;
+
+  public HiveOutputFileFactory(PartitionSpec spec, FileFormat fileFormat, LocationProvider locationProvider, FileIO io,
+                               EncryptionManager encryptionManager, TaskAttemptID taskAttemptID) {
+    super(spec, fileFormat, locationProvider, io, encryptionManager, 0, 0);
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  /**
+   * Override the filename generation so it contains jobId, taskId, taskAttemptId. Kept the UUID and the fileCount so
+   * the filenames are similar for other writers.
+   * @return The generated file name
+   */
+  @Override
+  protected String generateFilename() {
+    return format().addExtension(
+        String.format("%05d-%d-%d-%s-%05d", taskAttemptID.getJobID().getId(), taskAttemptID.getTaskID().getId(),

Review comment:
       The uuid is generated by the OutputFileFactory - this will be generated on every mapper-reducer. So different tasks / even task attempts will have a different uuid. The only thing connecting together the files generated by the same query would be the jobID (and even that can restart).
   I was thinking about changing the fileName format to something like:
   `<queryId>-<jobId>-<taskId>-<attemptId>-<fileCount>.<fileFormat>`
   But decided to keep something more like the current names for the familiarities shake.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535479061



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Preconditions.checkArgument(row instanceof Container);

Review comment:
       Because in the next line we try to cast it to a Container? 😄 
   More seriously: Previous MR patches introduced Container as a way to convert Iceberg Records to Writables and this is needed to push them through the MR framework. HiveIcebergRecordWriter only able to handle Containers and to be even more precise Container<Record> objects.
   
   After you made me think more, I got rid of this method call as the next line should throw an exception anyway and this check did not provide any more information above a ClassCastException. Also this is called for every record so saving anything small here could result in big additional performance gains.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r538112251



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveOutputFileFactory.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.LocationProvider;
+import org.apache.iceberg.io.OutputFileFactory;
+
+public class HiveOutputFileFactory extends OutputFileFactory {
+  private final TaskAttemptID taskAttemptID;
+
+  public HiveOutputFileFactory(PartitionSpec spec, FileFormat fileFormat, LocationProvider locationProvider, FileIO io,
+                               EncryptionManager encryptionManager, TaskAttemptID taskAttemptID) {
+    super(spec, fileFormat, locationProvider, io, encryptionManager, 0, 0);
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  /**
+   * Override the filename generation so it contains jobId, taskId, taskAttemptId. Kept the UUID and the fileCount so
+   * the filenames are similar for other writers.
+   * @return The generated file name
+   */
+  @Override
+  protected String generateFilename() {
+    return format().addExtension(
+        String.format("%05d-%d-%d-%s-%05d", taskAttemptID.getJobID().getId(), taskAttemptID.getTaskID().getId(),

Review comment:
       Removed HiveOutputFileFactory - added new cosntructor to OutputFileFactory instead
   
   Also since the locations are generated by the OutputFileFactory, we do not need LocationHelper anymore, so I removed it as well.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-740138355


   Looks like the last update accidentally included a few .crc files that are temp data from tests. Could you clean up the additions?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536910823



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();

Review comment:
       Not a blocker since this is scoped to appends only, but can we detect when the user called `INSERT OVERWRITE` at least?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537542502



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);
+      } else {
+        LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job.
+    // Intentionally used foreach on a single item. Using the Tasks API here only for the retry capability.
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {

Review comment:
       Serialized FileIO and started to use this




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-743186155


   Thanks for the review and the merge @rdblue! I have learned a lot during the review!


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537543325



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);
+      } else {
+        LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {

Review comment:
       Refactored out the way of getting the `dataFiles`, and used it for removing the data files when the job is aborted




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537816713



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+  private FileIO io;
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  /**
+   * Collects the generated data files and creates a commit file storing the data file list.
+   * @param context The job context
+   * @throws IOException Thrown if there is an error writing the commit file
+   */
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(),
+        attemptID.getJobID(), attemptID.getTaskID().getId());
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    DataFile[] closedFiles;
+    if (writer != null) {
+      closedFiles = writer.complete();
+    } else {
+      closedFiles = new DataFile[0];
+    }
+
+    // Creating the file containing the data files generated by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, io(context.getJobConf()));
+  }
+
+  /**
+   * Removes files generated by this task.
+   * @param context The task context
+   * @throws IOException Thrown if there is an error closing the writer
+   */
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  /**
+   * Reads the commit files stored in the temp directory and collects the generated committed data files.
+   * Appends the data files to the table. At the end removes the temporary directory.
+   * @param jobContext The job context
+   */
+  @Override
+  public void commitJob(JobContext jobContext) {
+    JobConf conf = jobContext.getJobConf();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()));
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), true);
+
+    if (dataFiles.size() > 0) {
+      // Appending data files to the table
+      AppendFiles append = table.newAppend();
+      dataFiles.forEach(append::appendFile);
+      append.commit();
+      LOG.info("Commit took {} ms for table: {} with {} file(s)", System.currentTimeMillis() - startTime, table,
+          dataFiles.size());
+      LOG.debug("Added files {}", dataFiles);
+    } else {
+      LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Removes the generated data files, if there is a commit file already generated for them.
+   * The cleanup at the end removes the temporary directory as well.
+   * @param jobContext The job context
+   * @param status The status of the job
+   */
+  @Override
+  public void abortJob(JobContext jobContext, int status) {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), false);
+
+    // Check if we have files already committed and remove data files if there are any
+    if (dataFiles.size() > 0) {
+      Tasks.foreach(dataFiles)
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exc) -> LOG.debug("Failed on to remove data file {} on abort job", file.path(), exc))
+          .run(file -> io(jobContext.getJobConf()).deleteFile(file.path().toString()));

Review comment:
       Removed the `io(Configuration configuration)` method. Easier to understand and we do not really reuse the io.
   Done




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861


   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-739956963


   Big thanks @rdblue for reviewing the PR even on Saturday! Really appreciate it!
   
   Your comments were really useful as usual. Updated the PR so if you have time again I would love to hear you thoughts.
   
   Thanks,
   Peter


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537548400



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,

Review comment:
       Hive uses the `mapred` hadoop interface. Extended the `mapreduce` writer only to try to keep parity with the `mapreduce` package as well, but since it complicates stuff too much simply changed to extend `PartitionedFanoutWriter` instead.
   Moved to `PartitionedFanoutWriter` because without extra settings it is not guaranteed that the data is grouped by the partition key.
   




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537794216



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.io.PartitionedFanoutWriter;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends PartitionedFanoutWriter<Record>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container<Record>> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  private final FileIO io;
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static final Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  static HiveIcebergRecordWriter removeWriter(TaskAttemptID taskAttemptID) {
+    return writers.remove(taskAttemptID);
+  }
+
+  HiveIcebergRecordWriter(Schema schema, PartitionSpec spec, FileFormat format,
+      FileAppenderFactory<Record> appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize,
+      TaskAttemptID taskAttemptID) {
+    super(spec, format, appenderFactory, fileFactory, io, targetFileSize);
+    this.io = io;
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+  }
+
+  @Override
+  protected PartitionKey partition(Record row) {
+    currentKey.partition(row);
+    return currentKey;
+  }
+
+  @Override
+  public void write(Writable row) throws IOException {
+    super.write(((Container<Record>) row).get());
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) throws IOException {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    DataFile[] dataFiles = super.complete();

Review comment:
       This was just changed in #1867. I think you just need to update it to `super.dataFiles()`.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535469757



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Preconditions.checkArgument(row instanceof Container);
+
+    Record record = ((Container<Record>) row).get();
+
+    currentKey.partition(record);
+
+    AppenderWrapper currentAppender = openAppenders.get(currentKey);
+    if (currentAppender == null) {
+      currentAppender = getAppender();
+      openAppenders.put(currentKey.copy(), currentAppender);
+    }
+
+    currentAppender.appender.add(record);
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    // Close the open appenders and store the closed file data
+    for (PartitionKey key : openAppenders.keySet()) {
+      AppenderWrapper wrapper = openAppenders.get(key);
+      wrapper.close();
+      closedFileData.put(key,
+          new ClosedFileData(key, wrapper.location, fileFormat, wrapper.length(), wrapper.metrics()));
+    }
+
+    openAppenders.clear();
+
+    // If abort then remove the unnecessary files
+    if (abort) {
+      Tasks.foreach(closedFileData.values().stream().map(ClosedFileData::fileName).iterator())
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exception) -> LOG.debug("Failed on to remove file {} on abort", file, exception))
+          .run(io::deleteFile);
+    }
+    LOG.info("IcebergRecordWriter is closed. Created {} files", closedFileData);

Review comment:
       First I wanted to print the size, later decided to print the map.
   After a second look I have realized that the keys in the map are already contained in the values in the map. So ended up changing it to printing the values only




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536909737



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)

Review comment:
       I also like to explicitly set the failure behavior. This should probably use `throwFailureWhenFinished()` and `stopOnFailure()` because this can't continue if any task fails.
   
   Not using `stopOnFailure` is for tasks like cleaning up files, where each task should at least attempt.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537800829



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveOutputFileFactory.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.LocationProvider;
+import org.apache.iceberg.io.OutputFileFactory;
+
+public class HiveOutputFileFactory extends OutputFileFactory {
+  private final TaskAttemptID taskAttemptID;
+
+  public HiveOutputFileFactory(PartitionSpec spec, FileFormat fileFormat, LocationProvider locationProvider, FileIO io,
+                               EncryptionManager encryptionManager, TaskAttemptID taskAttemptID) {
+    super(spec, fileFormat, locationProvider, io, encryptionManager, 0, 0);
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  /**
+   * Override the filename generation so it contains jobId, taskId, taskAttemptId. Kept the UUID and the fileCount so
+   * the filenames are similar for other writers.
+   * @return The generated file name
+   */
+  @Override
+  protected String generateFilename() {
+    return format().addExtension(
+        String.format("%05d-%d-%d-%s-%05d", taskAttemptID.getJobID().getId(), taskAttemptID.getTaskID().getId(),

Review comment:
       The first number is typically the task ID so we can trace back to the task that produced a file easily.
   
   The job ID is constant and will be 1-to-1 with a write UUID, so I'm not sure that it is needed. If not, then this could use the existing factory and pass `taskAttemptId.getTaskID()` and `taskAttemptId.getId()` to the constructor (instead of 0, 0).
   
   I think I'd prefer using the existing one and not creating a new subclass here.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536913476



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Record record = ((Container<Record>) row).get();
+
+    // Update the current key with the record, so we do not create a new object for every record
+    currentKey.partition(record);
+
+    AppenderWrapper currentAppender = openAppenders.get(currentKey);
+    if (currentAppender == null) {
+      currentAppender = getAppender();
+      openAppenders.put(currentKey.copy(), currentAppender);
+    }
+
+    currentAppender.appender.add(record);
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    // Close the open appenders and store the closed file data
+    for (PartitionKey key : openAppenders.keySet()) {
+      AppenderWrapper wrapper = openAppenders.get(key);
+      wrapper.close();
+      closedFileData.put(key,
+          new ClosedFileData(key, wrapper.location, fileFormat, wrapper.length(), wrapper.metrics()));
+    }
+
+    openAppenders.clear();
+
+    // If abort then remove the unnecessary files
+    if (abort) {

Review comment:
       I would expect a log message for aborting this task, not just the close message below with how many files were created.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536909022



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();

Review comment:
       I don't think that we expect `DataFile` instances to be duplicated anywhere, so the set is probably unnecessary compared to using a `List`. Also, `DataFile` doesn't require that `equals` is implemented, so there is no guarantee that this would even deduplicate if there were duplicate data files.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536910471



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);

Review comment:
       I would probably remove added files from the log. Maybe a debug log if you think they would be helpful.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537543607



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/LocationHelper.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+class LocationHelper {
+  private static final String FOR_COMMIT_EXTENSION = ".forCommit";
+
+  private LocationHelper() {
+  }
+
+  /**
+   * Generates query directory location based on the configuration.
+   * Currently it uses tableLocation/queryId
+   * @param conf The job's configuration
+   * @return The directory to store the query result files
+   */
+  static String generateQueryLocation(Configuration conf) {
+    String tableLocation = conf.get(InputFormatConfig.TABLE_LOCATION);
+    String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    return tableLocation + "/" + queryId;

Review comment:
       Added `temp` to the table location




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537818514



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java
##########
@@ -133,4 +146,93 @@ public DecomposedPredicate decomposePredicate(JobConf jobConf, Deserializer dese
     predicate.pushedPredicate = (ExprNodeGenericFuncDesc) exprNodeDesc;
     return predicate;
   }
+
+  /**
+   * Returns the Table FileIO serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table FileIO object
+   */
+  public static FileIO io(Configuration configuration) {
+    return (FileIO) get(configuration, InputFormatConfig.FILE_IO);
+  }
+
+  /**
+   * Returns the Table LocationProvider serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table LocationProvider object
+   */
+  public static LocationProvider location(Configuration configuration) {
+    return (LocationProvider) get(configuration, InputFormatConfig.LOCATION_PROVIDER);
+  }
+
+  /**
+   * Returns the Table EncryptionManager serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table EncryptionManager object
+   */
+  public static EncryptionManager encryption(Configuration configuration) {
+    return (EncryptionManager) get(configuration, InputFormatConfig.ENCRYPTION_MANAGER);
+  }
+
+  /**
+   * Returns the Table Schema serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table Schema object
+   */
+  public static Schema schema(Configuration configuration) {
+    return SchemaParser.fromJson(configuration.get(InputFormatConfig.TABLE_SCHEMA));
+  }
+
+  /**
+   * Returns the Table PartitionSpec serialized to the configuration.
+   * @param configuration The configuration used to get the data from
+   * @return The Table PartitionSpec object
+   */
+  public static PartitionSpec spec(Configuration configuration) {
+    return PartitionSpecParser.fromJson(schema(configuration), configuration.get(InputFormatConfig.PARTITION_SPEC));
+  }
+
+  /**
+   * Stores the serializable table data in the configuration.
+   * Currently the following is handled:
+   * <ul>
+   *   <li>- Location</li>
+   *   <li>- Schema</li>
+   *   <li>- Partition specification</li>
+   *   <li>- FileIO for handling table files</li>
+   *   <li>- Location provider used for file generation</li>
+   *   <li>- Encryption manager for encryption handling</li>
+   * </ul>
+   * @param configuration The target configuration to store to
+   * @param table The table which we want to store to the configuration
+   */
+  @VisibleForTesting
+  static void put(Configuration configuration, Table table) {
+    configuration.set(InputFormatConfig.TABLE_LOCATION, table.location());
+    configuration.set(InputFormatConfig.TABLE_SCHEMA, SchemaParser.toJson(table.schema()));
+    configuration.set(InputFormatConfig.PARTITION_SPEC, PartitionSpecParser.toJson(table.spec()));
+
+    put(configuration, InputFormatConfig.FILE_IO, table.io());
+    put(configuration, InputFormatConfig.LOCATION_PROVIDER, table.locationProvider());
+    put(configuration, InputFormatConfig.ENCRYPTION_MANAGER, table.encryption());
+  }
+
+  private static void put(Configuration configuration, String key, Serializable object) {
+    try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+         ObjectOutputStream oos = new ObjectOutputStream(baos)) {
+      oos.writeObject(object);
+      configuration.set(key, Base64.getEncoder().encodeToString(baos.toByteArray()));
+    } catch (IOException ioe) {
+      throw new RuntimeException(String.format("Error serializing %s to configuration", object), ioe);
+    }
+  }
+
+  private static Object get(Configuration configuration, String key) {

Review comment:
       Definitely!
   Removed the methods for serializing/deserializing! :)




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536907736



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/ClosedFileData.java
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.Serializable;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+/**
+ * Class for storing the data file properties which are needed for an Iceberg commit.
+ * <ul>
+ *   <li>Partition key
+ *   <li>File name
+ *   <li>File format
+ *   <li>File size
+ *   <li>Metrics
+ * </ul>
+ */
+final class ClosedFileData implements Serializable {

Review comment:
       Is this needed? The `GenericDataFile` and `GenericDeleteFile` classes are already `Serializable`. In Spark, the [task result class](https://github.com/apache/iceberg/blob/master/spark3/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java#L469-L479) has just a `DataFile` array.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-742489830


   @rdblue: Cleaned up the commits, and rebased the patch. If you have time could you please check out this PR?
   Thanks,
   Peter


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536910091



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/LocationHelper.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+class LocationHelper {
+  private static final String FOR_COMMIT_EXTENSION = ".forCommit";
+
+  private LocationHelper() {
+  }
+
+  /**
+   * Generates query directory location based on the configuration.
+   * Currently it uses tableLocation/queryId
+   * @param conf The job's configuration
+   * @return The directory to store the query result files
+   */
+  static String generateQueryLocation(Configuration conf) {
+    String tableLocation = conf.get(InputFormatConfig.TABLE_LOCATION);
+    String queryId = conf.get(HiveConf.ConfVars.HIVEQUERYID.varname);
+    return tableLocation + "/" + queryId;
+  }
+
+  /**
+   * Generates the job temp location based on the job configuration.
+   * Currently it uses QUERY_LOCATION/jobId.
+   * @param conf The job's configuration
+   * @param jobId The JobID for the task
+   * @return The file to store the results
+   */
+  static String generateJobLocation(Configuration conf, JobID jobId) {
+    return generateQueryLocation(conf) + "/" + jobId;
+  }
+
+  /**
+   * Generates datafile location based on the task configuration.
+   * Currently it uses QUERY_LOCATION/jobId/taskAttemptId.
+   * @param conf The job's configuration
+   * @param taskAttemptId The TaskAttemptID for the task
+   * @return The file to store the results
+   */
+  static String generateDataFileLocation(Configuration conf, TaskAttemptID taskAttemptId) {
+    return generateJobLocation(conf, taskAttemptId.getJobID()) + "/" + taskAttemptId.toString();
+  }
+
+  /**
+   * Generates file location based on the task configuration and a specific task id.
+   * This file will be used to store the data required to generate the Iceberg commit.
+   * Currently it uses QUERY_LOCATION/jobId/task-[0..numTasks).forCommit.
+   * @param conf The job's configuration
+   * @param jobId The jobId for the task
+   * @param taskId The taskId for the commit file
+   * @return The file to store the results
+   */
+  static String generateFileForCommitLocation(Configuration conf, JobID jobId, int taskId) {
+    return generateJobLocation(conf, jobId) + "/task-" + taskId + FOR_COMMIT_EXTENSION;
+  }
+
+  /**
+   * Generates file location location based on the task configuration.
+   * This file will be used to store the data required to generate the Iceberg commit.
+   * Currently it uses QUERY_LOCATION/jobId/task-[0..numTasks).forCommit.
+   * @param conf The job's configuration
+   * @param taskAttemptId The TaskAttemptID for the task
+   * @return The file to store the results
+   */
+  static String generateFileForCommitLocation(Configuration conf, TaskAttemptID taskAttemptId) {

Review comment:
       Minor: I would recommend not adding this method because it should be obvious in the committer when paths are the same. By having this convenience method, it looks like two different signatures and a bit more magic.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536913048



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,

Review comment:
       I think it would be quite a bit easier to implement this if you based it on the existing class shared by Flink and Spark, `BaseTaskWriter` and its children, `PartitionedWriter` and `UnpartitionedWriter`. Since this needs to extend the mapreduce writer class, you probably can't do that directly. But, you could create an inner class that extends, for example, `UnpartitionedWriter` that is actually used for the write.
   
   Using that as a base would make it easier to reach feature parity between Hive and the other engines. By using the existing `FileOutputFactory`, this would have encryption support. And it would have support for rolling new files at a target size.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537790136



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+  private FileIO io;
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  /**
+   * Collects the generated data files and creates a commit file storing the data file list.
+   * @param context The job context
+   * @throws IOException Thrown if there is an error writing the commit file
+   */
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(),
+        attemptID.getJobID(), attemptID.getTaskID().getId());
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    DataFile[] closedFiles;
+    if (writer != null) {
+      closedFiles = writer.complete();
+    } else {
+      closedFiles = new DataFile[0];
+    }
+
+    // Creating the file containing the data files generated by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, io(context.getJobConf()));
+  }
+
+  /**
+   * Removes files generated by this task.
+   * @param context The task context
+   * @throws IOException Thrown if there is an error closing the writer
+   */
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  /**
+   * Reads the commit files stored in the temp directory and collects the generated committed data files.
+   * Appends the data files to the table. At the end removes the temporary directory.
+   * @param jobContext The job context
+   */
+  @Override
+  public void commitJob(JobContext jobContext) {
+    JobConf conf = jobContext.getJobConf();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()));
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), true);
+
+    if (dataFiles.size() > 0) {
+      // Appending data files to the table
+      AppendFiles append = table.newAppend();
+      dataFiles.forEach(append::appendFile);
+      append.commit();
+      LOG.info("Commit took {} ms for table: {} with {} file(s)", System.currentTimeMillis() - startTime, table,
+          dataFiles.size());
+      LOG.debug("Added files {}", dataFiles);
+    } else {
+      LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Removes the generated data files, if there is a commit file already generated for them.
+   * The cleanup at the end removes the temporary directory as well.
+   * @param jobContext The job context
+   * @param status The status of the job
+   */
+  @Override
+  public void abortJob(JobContext jobContext, int status) {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), false);
+
+    // Check if we have files already committed and remove data files if there are any
+    if (dataFiles.size() > 0) {
+      Tasks.foreach(dataFiles)
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exc) -> LOG.debug("Failed on to remove data file {} on abort job", file.path(), exc))
+          .run(file -> io(jobContext.getJobConf()).deleteFile(file.path().toString()));
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Cleans up the jobs temporary location.
+   * @param jobContext The job context
+   */
+  private void cleanup(JobContext jobContext) {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Cleaning for job: {} on location: {}", jobContext.getJobID(), location);
+
+    // Remove the job's temp directory recursively.
+    // Intentionally used foreach on a single item. Using the Tasks API here only for the retry capability.
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on cleanup job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true);
+          } catch (IOException e) {
+            throw new UncheckedIOException(String.format("Failed to delete job directory: %s", file), e);

Review comment:
       You can use `run(file -> {...}, IOException.class)` to be able to throw `IOException` from the callable here instead of catching it. That way you don't need the try/catch.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536913048



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,

Review comment:
       I think it would be quite a bit easier to implement this if you based it on the existing class shared by Flink and Spark, `BaseTaskWriter` and its children, `PartitionedWriter` and `UnpartitionedWriter`. Since this needs to extend the mapreduce writer class, you probably can't do that directly. But, you could create an inner class that extends, for example, `UnpartitionedWriter` that is actually used for the write.
   
   Using that as a base would make it easier to reach feature parity between Hive and the other engines. By using the existing `FileOutputFactory`, this would have encryption support. And it would have support for rolling new files at a target size. And also support for paths from `LocationProvider`.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535479523



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",
+            System.currentTimeMillis() - startTime, table, addedFiles);
+      } else {
+        LOG.info("Commit for Iceberg write taken {} ms for {} with no new files",
+            System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job
+    Tasks.foreach(location)

Review comment:
       Yes. Tasks are used for the retry here.
   Updated the comment so it is easier to understand




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537848771



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.io.PartitionedFanoutWriter;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends PartitionedFanoutWriter<Record>

Review comment:
       > What I see as a problem here is that the data should be sorted, or at least grouped by partition key, which I think is a non-trivial task without changing the query.
   
   Yes, and we still don't have this working in Spark yet. But we've documented how to write to tables.
   
   I'm okay with this for initial development, but I don't think it is something that we should ship with by default if we consider Hive writes production-ready.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537549729



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Record record = ((Container<Record>) row).get();
+
+    // Update the current key with the record, so we do not create a new object for every record
+    currentKey.partition(record);
+
+    AppenderWrapper currentAppender = openAppenders.get(currentKey);
+    if (currentAppender == null) {
+      currentAppender = getAppender();
+      openAppenders.put(currentKey.copy(), currentAppender);
+    }
+
+    currentAppender.appender.add(record);
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    // Close the open appenders and store the closed file data
+    for (PartitionKey key : openAppenders.keySet()) {
+      AppenderWrapper wrapper = openAppenders.get(key);
+      wrapper.close();
+      closedFileData.put(key,
+          new ClosedFileData(key, wrapper.location, fileFormat, wrapper.length(), wrapper.metrics()));
+    }
+
+    openAppenders.clear();
+
+    // If abort then remove the unnecessary files
+    if (abort) {

Review comment:
       Yeah, somehow missed that.
   Thanks for pointing out!
   Added the log message.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends org.apache.hadoop.mapreduce.RecordWriter<NullWritable, Container>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  private final FileIO io;
+  private final String location;
+  private final FileFormat fileFormat;
+  private final GenericAppenderFactory appenderFactory;
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  // Data for every partition is written to a different appender
+  // This map stores the open appenders for the given partition key
+  private final Map<PartitionKey, AppenderWrapper> openAppenders = new HashMap<>();
+  // When the appenders are closed the file data needed for the Iceberg commit is stored and accessible through
+  // this map
+  private final Map<PartitionKey, ClosedFileData> closedFileData = new HashMap<>();
+
+  HiveIcebergRecordWriter(TaskAttemptID taskAttemptID, Configuration conf, String location, FileFormat fileFormat,
+                          Schema schema, PartitionSpec spec) {
+    this.io = new HadoopFileIO(conf);
+    this.location = location;
+    this.fileFormat = fileFormat;
+    this.appenderFactory = new GenericAppenderFactory(schema);
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+    LOG.info("IcebergRecordWriter is created in {} with {}", location, fileFormat);
+  }
+
+  @Override
+  public void write(Writable row) {
+    Record record = ((Container<Record>) row).get();
+
+    // Update the current key with the record, so we do not create a new object for every record
+    currentKey.partition(record);
+
+    AppenderWrapper currentAppender = openAppenders.get(currentKey);
+    if (currentAppender == null) {
+      currentAppender = getAppender();
+      openAppenders.put(currentKey.copy(), currentAppender);
+    }
+
+    currentAppender.appender.add(record);
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    // Close the open appenders and store the closed file data
+    for (PartitionKey key : openAppenders.keySet()) {
+      AppenderWrapper wrapper = openAppenders.get(key);
+      wrapper.close();
+      closedFileData.put(key,
+          new ClosedFileData(key, wrapper.location, fileFormat, wrapper.length(), wrapper.metrics()));
+    }
+
+    openAppenders.clear();
+
+    // If abort then remove the unnecessary files
+    if (abort) {
+      Tasks.foreach(closedFileData.values().stream().map(ClosedFileData::fileName).iterator())
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exception) -> LOG.debug("Failed on to remove file {} on abort", file, exception))
+          .run(io::deleteFile);
+    }
+    LOG.info("IcebergRecordWriter is closed. Created {} files", closedFileData.values());
+  }
+
+  @Override
+  public void close(org.apache.hadoop.mapreduce.TaskAttemptContext context) throws IOException {
+    close(false);
+  }
+
+  @Override
+  public void close(Reporter reporter) throws IOException {
+    close(false);
+  }
+
+  public Set<ClosedFileData> closedFileData() {
+    return new HashSet<>(closedFileData.values());
+  }
+
+  static HiveIcebergRecordWriter removeWriter(TaskAttemptID taskAttemptID) {
+    return writers.remove(taskAttemptID);
+  }

Review comment:
       Done




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r535463345



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateToCommitFileLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readToCommitFile(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.length())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit for Iceberg write taken {} ms for {} with file(s) {}",
+            System.currentTimeMillis() - startTime, table, addedFiles);
+      } else {
+        LOG.info("Commit for Iceberg write taken {} ms for {} with no new files",
+            System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true /* recursive */);
+          } catch (IOException e) {
+            throw new RuntimeIOException(e, "Failed to delete job directory: %s", file);
+          }
+        });
+    cleanupJob(jobContext);
+  }
+
+  private static void createToCommitFile(Set<ClosedFileData> closedFiles, String location, FileIO io)
+      throws IOException {
+
+    OutputFile commitFile = io.newOutputFile(location);
+    ObjectOutputStream oos = new ObjectOutputStream(commitFile.createOrOverwrite());

Review comment:
       Good catch!
   Thanks




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536911121



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Creating the file containing the descriptor(s) for the file(s) written by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}, expecting {} file(s).", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)
+          .run(taskId -> {
+            String taskFileName = LocationHelper.generateFileForCommitLocation(conf, jobContext.getJobID(), taskId);
+            Set<ClosedFileData> closedFiles = readFileForCommit(taskFileName, table.io());
+
+            // If the data is not empty add to the table
+            if (!closedFiles.isEmpty()) {
+              closedFiles.forEach(file -> {
+                DataFiles.Builder builder = DataFiles.builder(table.spec())
+                    .withPath(file.fileName())
+                    .withFormat(file.fileFormat())
+                    .withFileSizeInBytes(file.fileSize())
+                    .withPartition(file.partitionKey())
+                    .withMetrics(file.metrics());
+                dataFiles.add(builder.build());
+              });
+            }
+          });
+
+      if (dataFiles.size() > 0) {
+        // Appending data files to the table
+        AppendFiles append = table.newAppend();
+        Set<String> addedFiles = new HashSet<>(dataFiles.size());
+        dataFiles.forEach(dataFile -> {
+          append.appendFile(dataFile);
+          addedFiles.add(dataFile.path().toString());
+        });
+        append.commit();
+        LOG.info("Commit took {} ms for table: {} with file(s): {}", System.currentTimeMillis() - startTime, table,
+            addedFiles);
+      } else {
+        LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+      }
+
+      // Calling super to cleanupJob if something more is needed
+      cleanupJob(jobContext);
+
+    } finally {
+      if (executor != null) {
+        executor.shutdown();
+      }
+    }
+  }
+
+  @Override
+  public void abortJob(JobContext jobContext, int status) throws IOException {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    // Remove the result directory for the failed job.
+    // Intentionally used foreach on a single item. Using the Tasks API here only for the retry capability.
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on abort job", file, exc))
+        .run(file -> {

Review comment:
       It would be nice to do this with the `FileIO` API rather than using Hadoop. Since each task generates just one commit file, this could generate the commit file location and use `io.deleteFile(commitFile)` right?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-740142982


   > Looks like the last update accidentally included a few .crc files that are temp data from tests. Could you clean up the additions?
   
   Of course. I have removed the extra files.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537816859



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergRecordWriter.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileAppenderFactory;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.io.PartitionedFanoutWriter;
+import org.apache.iceberg.mr.mapred.Container;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+class HiveIcebergRecordWriter extends PartitionedFanoutWriter<Record>
+    implements FileSinkOperator.RecordWriter, org.apache.hadoop.mapred.RecordWriter<NullWritable, Container<Record>> {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergRecordWriter.class);
+
+  // The current key is reused at every write to avoid unnecessary object creation
+  private final PartitionKey currentKey;
+  private final FileIO io;
+
+  // <TaskAttemptId, HiveIcebergRecordWriter> map to store the active writers
+  // Stored in concurrent map, since some executor engines can share containers
+  private static final Map<TaskAttemptID, HiveIcebergRecordWriter> writers = new ConcurrentHashMap<>();
+
+  static HiveIcebergRecordWriter removeWriter(TaskAttemptID taskAttemptID) {
+    return writers.remove(taskAttemptID);
+  }
+
+  HiveIcebergRecordWriter(Schema schema, PartitionSpec spec, FileFormat format,
+      FileAppenderFactory<Record> appenderFactory, OutputFileFactory fileFactory, FileIO io, long targetFileSize,
+      TaskAttemptID taskAttemptID) {
+    super(spec, format, appenderFactory, fileFactory, io, targetFileSize);
+    this.io = io;
+    this.currentKey = new PartitionKey(spec, schema);
+    writers.put(taskAttemptID, this);
+  }
+
+  @Override
+  protected PartitionKey partition(Record row) {
+    currentKey.partition(row);
+    return currentKey;
+  }
+
+  @Override
+  public void write(Writable row) throws IOException {
+    super.write(((Container<Record>) row).get());
+  }
+
+  @Override
+  public void write(NullWritable key, Container value) throws IOException {
+    write(value);
+  }
+
+  @Override
+  public void close(boolean abort) throws IOException {
+    DataFile[] dataFiles = super.complete();

Review comment:
       Fixed
   Thanks for the pointer!




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537791648



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+  private FileIO io;
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  /**
+   * Collects the generated data files and creates a commit file storing the data file list.
+   * @param context The job context
+   * @throws IOException Thrown if there is an error writing the commit file
+   */
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(),
+        attemptID.getJobID(), attemptID.getTaskID().getId());
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    DataFile[] closedFiles;
+    if (writer != null) {
+      closedFiles = writer.complete();
+    } else {
+      closedFiles = new DataFile[0];
+    }
+
+    // Creating the file containing the data files generated by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, io(context.getJobConf()));
+  }
+
+  /**
+   * Removes files generated by this task.
+   * @param context The task context
+   * @throws IOException Thrown if there is an error closing the writer
+   */
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  /**
+   * Reads the commit files stored in the temp directory and collects the generated committed data files.
+   * Appends the data files to the table. At the end removes the temporary directory.
+   * @param jobContext The job context
+   */
+  @Override
+  public void commitJob(JobContext jobContext) {
+    JobConf conf = jobContext.getJobConf();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()));
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), true);
+
+    if (dataFiles.size() > 0) {
+      // Appending data files to the table
+      AppendFiles append = table.newAppend();
+      dataFiles.forEach(append::appendFile);
+      append.commit();
+      LOG.info("Commit took {} ms for table: {} with {} file(s)", System.currentTimeMillis() - startTime, table,
+          dataFiles.size());
+      LOG.debug("Added files {}", dataFiles);
+    } else {
+      LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Removes the generated data files, if there is a commit file already generated for them.
+   * The cleanup at the end removes the temporary directory as well.
+   * @param jobContext The job context
+   * @param status The status of the job
+   */
+  @Override
+  public void abortJob(JobContext jobContext, int status) {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), false);
+
+    // Check if we have files already committed and remove data files if there are any
+    if (dataFiles.size() > 0) {
+      Tasks.foreach(dataFiles)
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exc) -> LOG.debug("Failed on to remove data file {} on abort job", file.path(), exc))
+          .run(file -> io(jobContext.getJobConf()).deleteFile(file.path().toString()));

Review comment:
       Minor: `io` stores the `FileIO` after it is loaded, but it still seems odd to call the method and pass in the configuration each time. I think it would be cleaner to have `FileIO = io(jobContext.getJobConf());` at the top of this method. That keeps the handler here simpler.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] pvary commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r537816181



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.UncheckedIOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+  private FileIO io;
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  /**
+   * Collects the generated data files and creates a commit file storing the data file list.
+   * @param context The job context
+   * @throws IOException Thrown if there is an error writing the commit file
+   */
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String fileForCommitLocation = LocationHelper.generateFileForCommitLocation(context.getJobConf(),
+        attemptID.getJobID(), attemptID.getTaskID().getId());
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    DataFile[] closedFiles;
+    if (writer != null) {
+      closedFiles = writer.complete();
+    } else {
+      closedFiles = new DataFile[0];
+    }
+
+    // Creating the file containing the data files generated by this task
+    createFileForCommit(closedFiles, fileForCommitLocation, io(context.getJobConf()));
+  }
+
+  /**
+   * Removes files generated by this task.
+   * @param context The task context
+   * @throws IOException Thrown if there is an error closing the writer
+   */
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  /**
+   * Reads the commit files stored in the temp directory and collects the generated committed data files.
+   * Appends the data files to the table. At the end removes the temporary directory.
+   * @param jobContext The job context
+   */
+  @Override
+  public void commitJob(JobContext jobContext) {
+    JobConf conf = jobContext.getJobConf();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job has started for table: {}, using location: {}", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()));
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), true);
+
+    if (dataFiles.size() > 0) {
+      // Appending data files to the table
+      AppendFiles append = table.newAppend();
+      dataFiles.forEach(append::appendFile);
+      append.commit();
+      LOG.info("Commit took {} ms for table: {} with {} file(s)", System.currentTimeMillis() - startTime, table,
+          dataFiles.size());
+      LOG.debug("Added files {}", dataFiles);
+    } else {
+      LOG.info("Commit took {} ms for table: {} with no new files", System.currentTimeMillis() - startTime, table);
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Removes the generated data files, if there is a commit file already generated for them.
+   * The cleanup at the end removes the temporary directory as well.
+   * @param jobContext The job context
+   * @param status The status of the job
+   */
+  @Override
+  public void abortJob(JobContext jobContext, int status) {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Job {} is aborted. Cleaning job location {}", jobContext.getJobID(), location);
+
+    List<DataFile> dataFiles = dataFiles(jobContext, io(jobContext.getJobConf()), false);
+
+    // Check if we have files already committed and remove data files if there are any
+    if (dataFiles.size() > 0) {
+      Tasks.foreach(dataFiles)
+          .retry(3)
+          .suppressFailureWhenFinished()
+          .onFailure((file, exc) -> LOG.debug("Failed on to remove data file {} on abort job", file.path(), exc))
+          .run(file -> io(jobContext.getJobConf()).deleteFile(file.path().toString()));
+    }
+
+    cleanup(jobContext);
+  }
+
+  /**
+   * Cleans up the jobs temporary location.
+   * @param jobContext The job context
+   */
+  private void cleanup(JobContext jobContext) {
+    String location = LocationHelper.generateJobLocation(jobContext.getJobConf(), jobContext.getJobID());
+    LOG.info("Cleaning for job: {} on location: {}", jobContext.getJobID(), location);
+
+    // Remove the job's temp directory recursively.
+    // Intentionally used foreach on a single item. Using the Tasks API here only for the retry capability.
+    Tasks.foreach(location)
+        .retry(3)
+        .suppressFailureWhenFinished()
+        .onFailure((file, exc) -> LOG.debug("Failed on to remove directory {} on cleanup job", file, exc))
+        .run(file -> {
+          Path toDelete = new Path(file);
+          FileSystem fs = Util.getFs(toDelete, jobContext.getJobConf());
+          try {
+            fs.delete(toDelete, true);
+          } catch (IOException e) {
+            throw new UncheckedIOException(String.format("Failed to delete job directory: %s", file), e);

Review comment:
       Thanks!
   Done




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#issuecomment-742712808


   @pvary, looks great! I merged this. Thanks for separating these into smaller PRs, it really helps keep the reviews manageable.


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1861: Hive: OutputCommitter implementation for Hive writes

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1861:
URL: https://github.com/apache/iceberg/pull/1861#discussion_r536909153



##########
File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.iceberg.mr.hive;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.OutputCommitter;
+import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.hadoop.HadoopFileIO;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.mr.Catalogs;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An Iceberg table committer for adding data files to the Iceberg tables.
+ * Currently independent of the Hive ACID transactions.
+ */
+public final class HiveIcebergOutputCommitter extends OutputCommitter {
+  private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergOutputCommitter.class);
+
+  @Override
+  public void setupJob(JobContext jobContext) {
+    // do nothing.
+  }
+
+  @Override
+  public void setupTask(TaskAttemptContext taskAttemptContext) {
+    // do nothing.
+  }
+
+  @Override
+  public boolean needsTaskCommit(TaskAttemptContext context) {
+    // We need to commit if this is the last phase of a MapReduce process
+    return TaskType.REDUCE.equals(context.getTaskAttemptID().getTaskID().getTaskType()) ||
+        context.getJobConf().getNumReduceTasks() == 0;
+  }
+
+  @Override
+  public void commitTask(TaskAttemptContext context) throws IOException {
+    TaskAttemptID attemptID = context.getTaskAttemptID();
+    String commitFileLocation = LocationHelper.generateToCommitFileLocation(context.getJobConf(), attemptID);
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(attemptID);
+
+    Set<ClosedFileData> closedFiles = Collections.emptySet();
+    if (writer != null) {
+      closedFiles = writer.closedFileData();
+    }
+
+    // Create the committed file for the task
+    createToCommitFile(closedFiles, commitFileLocation, new HadoopFileIO(context.getJobConf()));
+  }
+
+  @Override
+  public void abortTask(TaskAttemptContext context) throws IOException {
+    // Clean up writer data from the local store
+    HiveIcebergRecordWriter writer = HiveIcebergRecordWriter.removeWriter(context.getTaskAttemptID());
+
+    // Remove files if it was not done already
+    writer.close(true);
+  }
+
+  @Override
+  public void commitJob(JobContext jobContext) throws IOException {
+    JobConf conf = jobContext.getJobConf();
+    // If there are reducers, then every reducer will generate a result file.
+    // If this is a map only task, then every mapper will generate a result file.
+    int expectedFiles = conf.getNumReduceTasks() != 0 ? conf.getNumReduceTasks() : conf.getNumMapTasks();
+    Table table = Catalogs.loadTable(conf);
+
+    long startTime = System.currentTimeMillis();
+    LOG.info("Committing job is started for {} using {} expecting {} file(s)", table,
+        LocationHelper.generateJobLocation(conf, jobContext.getJobID()), expectedFiles);
+
+    ExecutorService executor = null;
+    try {
+      // Creating executor service for parallel handling of file reads
+      executor = Executors.newFixedThreadPool(
+          conf.getInt(InputFormatConfig.COMMIT_THREAD_POOL_SIZE, InputFormatConfig.COMMIT_THREAD_POOL_SIZE_DEFAULT),
+          new ThreadFactoryBuilder()
+              .setDaemon(true)
+              .setPriority(Thread.NORM_PRIORITY)
+              .setNameFormat("iceberg-commit-pool-%d")
+              .build());
+
+      Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet();
+
+      // Reading the committed files. The assumption here is that the taskIds are generated in sequential order
+      // starting from 0.
+      Tasks.range(expectedFiles)
+          .executeWith(executor)
+          .retry(3)

Review comment:
       +1 for avoiding extra configuration unless we have a use case for it later.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org