steveloughran commented on a change in pull request #1407: URL: https://github.com/apache/iceberg/pull/1407#discussion_r526239303
########## File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java ########## @@ -0,0 +1,208 @@ +/* + * 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 { Review comment: why this and not the (newer) org.apache.hadoop.mapreduce.OutputCommitter ########## File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java ########## @@ -0,0 +1,208 @@ +/* + * 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(); Review comment: Been having fun in spark and hadoop about making sure this is unique. What runtime are you targeting here, and where does it get job/task IDs? ########## File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java ########## @@ -0,0 +1,208 @@ +/* + * 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 = HiveIcebergOutputFormat.writers.remove(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 = HiveIcebergOutputFormat.writers.remove(context.getTaskAttemptID()); + + // Remove files if it was not done already + writer.close(true); + } + + @Override + public void commitJob(JobContext jobContext) throws IOException { Review comment: recommend a lot more logging here including * fact commit is starting, destination, job ID * at end: time to commit * when there is/isn't work to commit * directory scanned for files ########## File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java ########## @@ -0,0 +1,208 @@ +/* + * 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 = HiveIcebergOutputFormat.writers.remove(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 = HiveIcebergOutputFormat.writers.remove(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); + + 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(false) + .setPriority(Thread.NORM_PRIORITY) + .setNameFormat("iceberg-commit-pool-%d") + .build()); + + Set<DataFile> dataFiles = new ConcurrentHashMap<>().newKeySet(); Review comment: never seen this before. nice trick. ########## File path: mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java ########## @@ -0,0 +1,208 @@ +/* + * 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 = HiveIcebergOutputFormat.writers.remove(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 = HiveIcebergOutputFormat.writers.remove(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); + + 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(false) Review comment: maybe safer with daemon=true ---------------------------------------------------------------- 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: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
