[ 
https://issues.apache.org/jira/browse/MAPREDUCE-7341?focusedWorklogId=624475&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-624475
 ]

ASF GitHub Bot logged work on MAPREDUCE-7341:
---------------------------------------------

                Author: ASF GitHub Bot
            Created on: 19/Jul/21 16:26
            Start Date: 19/Jul/21 16:26
    Worklog Time Spent: 10m 
      Work Description: mukund-thakur commented on a change in pull request 
#2971:
URL: https://github.com/apache/hadoop/pull/2971#discussion_r672449812



##########
File path: 
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/ManifestCommitterConfig.java
##########
@@ -0,0 +1,335 @@
+/*
+ * 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 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.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+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.mapreduce.lib.output.FileOutputCommitter.SUCCESSFUL_JOB_OUTPUT_DIR_MARKER;
+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.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.ManifestCommitterSupport.buildJobUUID;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterSupport.getAppAttemptId;
+
+/**
+ * The configuration for the committer as built up from the job configuration
+ * and data passed down from the committer factory.
+ * Isolated for ease of dev/test
+ */
+final class ManifestCommitterConfig implements IOStatisticsSource {
+
+  /**
+   * Final destination of work.
+   * This is <i>unqualified</i>.
+   */
+  private final Path destinationDir;
+
+  /**
+   * Role: used in log/text messages.
+   */
+  private final String role;
+
+  /**
+   * This is the directory for all intermediate work: where the output
+   * format will write data.
+   */
+  private final Path taskAttemptDir;
+
+  /** Configuration of the job. */
+  private final Configuration conf;
+
+  /** The job context. For a task, this can be cast to a TaskContext. */
+  private final JobContext jobContext;
+
+  /** Should a job marker be created? */
+  private final boolean createJobMarker;
+
+  /**
+   * Job ID Or UUID -without any attempt suffix.
+   * This is expected/required to be unique, though
+   * Spark has had "issues" there until recently
+   * with lack of uniqueness of generated MR Job IDs.
+   */
+  private final String jobUniqueId;
+
+  /**
+   * Where did the job Unique ID come from?
+   */
+  private final String jobUniqueIdSource;
+
+  /**
+   * Number of this attempt; starts at zero.
+   */
+  private final int jobAttemptNumber;
+
+  /**
+   * Job ID + AttemptID.
+   */
+  private final String jobAttemptId;
+
+  /**
+   * Task ID: used as the filename of the manifest.
+   */
+  private final String taskId;
+
+  /**
+   * Task attempt ID. Determines the working
+   * directory for task attempts to write data into,
+   * and for the task committer to scan.
+   */
+  private final String taskAttemptId;
+
+  /** Any progressable for progress callbacks. */
+  private final Progressable progressable;
+
+  /**
+   * IOStatistics to update.
+   */
+  private final IOStatisticsStore iostatistics;
+
+  /** Should the output be validated after the commit? */
+  private final boolean validateOutput;
+
+  /**
+   * Attempt directory management.
+   */
+  private final ManifestCommitterSupport.AttemptDirectories dirs;
+
+  /**
+   * Callback when a stage is entered.
+   */
+  private final StageEventCallbacks stageEventCallbacks;
+
+  /**
+   * Constructor.
+   * @param outputPath destination path of the job.
+   * @param role role for log messages.
+   * @param context job/task context
+   * @param iostatistics IO Statistics
+   * @param stageEventCallbacks stage event callbacks.
+   */
+  ManifestCommitterConfig(
+      final Path outputPath,
+      final String role,
+      final JobContext context,
+      final IOStatisticsStore iostatistics,
+      final StageEventCallbacks stageEventCallbacks) {
+    this.role = role;
+    this.jobContext = context;
+    this.conf = context.getConfiguration();
+    this.destinationDir = outputPath;
+    this.iostatistics = iostatistics;
+    this.stageEventCallbacks = stageEventCallbacks;
+    this.createJobMarker = conf.getBoolean(
+        SUCCESSFUL_JOB_OUTPUT_DIR_MARKER,
+        DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER);
+    this.validateOutput = conf.getBoolean(
+        OPT_VALIDATE_OUTPUT,
+        OPT_VALIDATE_OUTPUT_DEFAULT);
+    Pair<String, String> pair = buildJobUUID(conf, context.getJobID());
+    this.jobUniqueId = pair.getLeft();
+    this.jobUniqueIdSource = pair.getRight();
+    this.jobAttemptNumber = getAppAttemptId(context);
+    this.jobAttemptId = this.jobUniqueId + "_" + jobAttemptNumber;
+
+    // build directories
+    this.dirs = new ManifestCommitterSupport.AttemptDirectories(outputPath,
+        this.jobUniqueId, jobAttemptNumber);
+
+    // if constructed with a task attempt, build the task ID and path.
+    if (context instanceof TaskAttemptContext) {
+      // it's a task
+      final TaskAttemptContext tac = (TaskAttemptContext) context;
+      TaskAttemptID taskAttempt = Objects.requireNonNull(
+          tac.getTaskAttemptID());
+      taskAttemptId = taskAttempt.toString();
+      taskId = taskAttempt.getTaskID().toString();
+      // Task attempt dir; must be different across instances
+      taskAttemptDir = dirs.getTaskAttemptPath(taskAttemptId);
+      // the context is also the progress callback.
+      progressable = tac;
+
+    } else {

Review comment:
       That will reduce the casting and null assignments. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org

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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 624475)
    Time Spent: 8h 50m  (was: 8h 40m)

> Add a task-manifest output committer for Azure and GCS
> ------------------------------------------------------
>
>                 Key: MAPREDUCE-7341
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7341
>             Project: Hadoop Map/Reduce
>          Issue Type: New Feature
>          Components: client
>    Affects Versions: 3.3.1
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 8h 50m
>  Remaining Estimate: 0h
>
> Add a task-manifest output committer for Azure and GCS
> The S3A committers are very popular in Spark on S3, as they are both correct 
> and fast.
> The classic FileOutputCommitter v1 and v2 algorithms are all that is 
> available for Azure ABFS and Google GCS, and they have limitations. 
> The v2 algorithm isn't safe in the presence of failed task attempt commits, 
> so we
> recommend the v1 algorithm for Azure. But that is slow because it 
> sequentially lists
> then renames files and directories, one-by-one. The latencies of list
> and rename make things slow.
> Google GCS lacks the atomic directory rename required for v1 correctness;
> v2 can be used (which doesn't have the job commit performance limitations),
> but it's not safe.
> Proposed
> * Add a new FileOutputFormat committer which uses an intermediate manifest to
>   pass the list of files created by a TA to the job committer.
> * Job committer to parallelise reading these task manifests and submit all the
>   rename operations into a pool of worker threads. (also: mkdir, directory 
> deletions on cleanup)
> * Use the committer plugin mechanism added for s3a to make this the default 
> committer for ABFS
>   (i.e. no need to make any changes to FileOutputCommitter)
> * Add lots of IOStatistics instrumentation + logging of operations in the 
> JobCommit
>   for visibility of where delays are occurring.
> * Reuse the S3A committer _SUCCESS JSON structure to publish IOStats & other 
> data
>   for testing/support.  
> This committer will be faster than the V1 algorithm because of the 
> parallelisation, and
> because a manifest written by create-and-rename will be exclusive to a single 
> task
> attempt, delivers the isolation which the v2 committer lacks.
> This is not an attempt to do an iceberg/hudi/delta-lake style manifest-only 
> format
> for describing the contents of a table; the final output is still a directory 
> tree
> which must be scanned during query planning.
> As such the format is still suboptimal for cloud storage -but at least we 
> will have
> faster job execution during the commit phases.
>   
> Note: this will also work on HDFS, where again, it should be faster than
> the v1 committer. However the target is very much Spark with ABFS and GCS; no 
> plans to worry about MR as that simplifies the challenge of dealing with job 
> restart (i.e. you don't have to)



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: mapreduce-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: mapreduce-issues-h...@hadoop.apache.org

Reply via email to