steveloughran commented on a change in pull request #2971: URL: https://github.com/apache/hadoop/pull/2971#discussion_r647789479
########## File path: hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitter.java ########## @@ -0,0 +1,712 @@ +/* + * 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.hadoop.mapreduce.lib.output.committer.manifest; + +import java.io.IOException; +import java.util.Objects; +import java.util.concurrent.ExecutorService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.TaskManifest; +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.concurrent.HadoopExecutors; + +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.logIOStatisticsAtDebug; +import static org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.CleanupJobStage.optionsFromConfig; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_SUFFIX; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_IO_PROCESSORS_DEFAULT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_VALIDATE_OUTPUT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.OPT_VALIDATE_OUTPUT_DEFAULT; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterStatisticNames.COMMITTER_TASKS_COMPLETED; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterSupport.buildJobUUID; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterSupport.createIOStatisticsStore; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterSupport.getAppAttemptId; + +/** + * This is the Intermediate-Manifest committer. + */ +public class ManifestCommitter extends PathOutputCommitter implements + IOStatisticsSource { + + public static final Logger LOG = LoggerFactory.getLogger( + ManifestCommitter.class); + + public static final String TASK_COMMITTER = "task committer"; + + public static final String JOB_COMMITTER = "job committer"; + + /** + * Committer Configuration as extracted from + * the job/task context and set in the constructor. + * + */ + private final ManifestCommitterConfig baseConfig; + + /** + * Destination of the job. + */ + private final Path destinationDir; + + /** + * For tasks, the attempt directory. + * Null for jobs. + */ + private final Path taskAttemptDir; + + /** + * IOStatistics to update. + */ + private final IOStatisticsStore iostatistics; + + /** + * The job Manifest Success data; only valid after a job successfully + * commits. + */ + private ManifestSuccessData jobSuccessData; + + /** + * The task manifest of the task commit. + * Null unless this is a task attempt and the + * task has successfully been committed. + */ + private TaskManifest taskAttemptCommittedManifest; + + /** + * Create a committer. + * @param outputPath output path + * @param context job/task context + * @throws IOException failure. + */ + ManifestCommitter(final Path outputPath, + final TaskAttemptContext context) throws IOException { + super(outputPath, context); + destinationDir = resolveDestinationDirectory(outputPath, + context.getConfiguration()); + iostatistics = createIOStatisticsStore().build(); + baseConfig = new ManifestCommitterConfig(destinationDir, + TASK_COMMITTER, + context, + iostatistics); + taskAttemptDir = baseConfig.getTaskAttemptDir(); + } + + /** + * Create a committer config from the passed in job/task context. + * @param isTask is this a task? + * @param context context + * @return committer config + */ + private ManifestCommitterConfig createCommitterConfig(boolean isTask, + JobContext context) { + return new ManifestCommitterConfig( + getOutputPath(), + isTask ? TASK_COMMITTER : JOB_COMMITTER, + context, + iostatistics); + } + + /** + * Set up a job through a {@link SetupJobStage}. + * @param jobContext Context of the job whose output is being written. + * @throws IOException IO Failure. + */ + public void setupJob(final JobContext jobContext) throws IOException { + ManifestCommitterConfig committerConfig = createCommitterConfig(false, + jobContext); + StageConfig stageConfig = + committerConfig + .createJobStageConfig() + .withOperations(createStoreOperations()) + .build(); + // set up the job. + new SetupJobStage(stageConfig) + .apply(committerConfig.getCreateJobMarker()); + logCommitterStatisticsAtDebug(); + } + + /** + * Set up a task through a {@link SetupTaskStage}. + * + * @param context task context. + * @throws IOException IO Failure. + */ + public void setupTask(final TaskAttemptContext context) + throws IOException { + StageConfig stageConfig = + createCommitterConfig(true, context) + .createJobStageConfig() + .withOperations(createStoreOperations()) + .build(); + // create task attempt dir; delete if present. Or fail? + new SetupTaskStage(stageConfig).apply(""); + logCommitterStatisticsAtDebug(); + } + + /** + * Always return true. + * @param context task context. + * @return true + * @throws IOException IO Failure. + */ + public boolean needsTaskCommit(final TaskAttemptContext context) + throws IOException { + return true; + } + + /** + * Failure during Job Commit is not recoverable from. + * + * @param jobContext + * Context of the job whose output is being written. + * @return false, always + * @throws IOException never + */ + @Override + public boolean isCommitJobRepeatable(final JobContext jobContext) + throws IOException { + return false; + } + + /** + * Declare that task recovery is not supported. + * It would be, if someone added the code *and tests*. + * @param jobContext + * Context of the job whose output is being written. + * @return false, always + * @throws IOException never + */ + @Override + public boolean isRecoverySupported(final JobContext jobContext) + throws IOException { + return false; + } + + /** + * + * @param taskContext Context of the task whose output is being recovered + * @throws IOException always + */ + @Override + public void recoverTask(final TaskAttemptContext taskContext) + throws IOException { + LOG.warn("Rejecting recoverTask({}) call", taskContext.getTaskAttemptID()); + throw new IOException("Cannot recover task " + + taskContext.getTaskAttemptID()); + } + + /** + * Commit the task. + * This is where the task attempt tree list takes place. + * @param context task context. + * @throws IOException IO Failure. + */ + public void commitTask(final TaskAttemptContext context) + throws IOException { + ManifestCommitterConfig committerConfig = createCommitterConfig(true, + context); + StageConfig stageConfig = committerConfig.createJobStageConfig() + .withOperations(createStoreOperations()) + .build(); + taskAttemptCommittedManifest = new CommitTaskStage(stageConfig) + .apply("generate") + .getRight(); + iostatistics.incrementCounter(COMMITTER_TASKS_COMPLETED, 1); + logCommitterStatisticsAtDebug(); + } + + /** + * Abort a task. + * @param context task context + * @throws IOException failure during the delete + */ + public void abortTask(final TaskAttemptContext context) + throws IOException { + ManifestCommitterConfig committerConfig = createCommitterConfig(true, + context); + new AbortTaskStage(committerConfig.createJobStageConfig() + .withOperations(createStoreOperations()) + .build()) + .apply(false); + logCommitterStatisticsAtDebug(); + } + + /** + * This is the big job commit stage. + * Load the manifests, prepare the destination, rename + * the files then cleanup the job directory. + * @param jobContext Context of the job whose output is being written. + * @throws IOException failure. + */ + @Override + public void commitJob(final JobContext jobContext) throws IOException { + + ManifestCommitterConfig committerConfig = createCommitterConfig(false, + jobContext); + + try (CloseableTaskSubmitter ioProcs = + committerConfig.createSubmitter()) { + // the stage config will be shared across all stages. + StageConfig stageConfig = committerConfig.createJobStageConfig() + .withOperations(createStoreOperations()) + .withIOProcessors(ioProcs) Review comment: I should be more explicit. Each stage is executed in sequence within a task commit or job commit method; the thread pool is local to the commitTask/commitJob methods -- 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]
