steveloughran commented on a change in pull request #2971:
URL: https://github.com/apache/hadoop/pull/2971#discussion_r819673642



##########
File path: 
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/impl/ManifestCommitterSupport.java
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.impl;
+
+import java.io.IOException;
+import java.time.ZonedDateTime;
+
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.EtagSource;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathIOException;
+import org.apache.hadoop.fs.statistics.IOStatisticsAggregator;
+import org.apache.hadoop.fs.statistics.IOStatisticsSource;
+import org.apache.hadoop.fs.statistics.impl.IOStatisticsStoreBuilder;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants;
+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.mapreduce.lib.output.committer.manifest.stages.StageConfig;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import static java.util.Objects.requireNonNull;
+import static 
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.iostatisticsStore;
+import static 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.PENDING_DIR_NAME;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.INITIAL_APP_ATTEMPT_ID;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ATTEMPT_DIR_FORMAT_STR;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_DIR_FORMAT_STR;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_ID_SOURCE_MAPREDUCE;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_ATTEMPT_SUBDIR;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.JOB_TASK_MANIFEST_SUBDIR;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.MANIFEST_COMMITTER_CLASSNAME;
+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_STORE_OPERATIONS_CLASS;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SPARK_WRITE_UUID;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.SUMMARY_FILENAME_FORMAT;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.ManifestCommitterConstants.TMP_SUFFIX;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.PRINCIPAL;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.COUNTER_STATISTICS;
+import static 
org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.InternalConstants.DURATION_STATISTICS;
+
+/**
+ * Class for manifest committer support util methods.
+ */
+public final class ManifestCommitterSupport {
+
+  private ManifestCommitterSupport() {
+  }
+
+  /**
+   * Create an IOStatistics Store with the standard statistics
+   * set up.
+   * @return a store builder preconfigured with the standard stats.
+   */
+  public static IOStatisticsStoreBuilder createIOStatisticsStore() {
+
+    final IOStatisticsStoreBuilder store
+        = iostatisticsStore();
+
+    store.withCounters(COUNTER_STATISTICS);
+    store.withMaximums(COUNTER_STATISTICS);
+    store.withMinimums(COUNTER_STATISTICS);
+    store.withMeanStatistics(COUNTER_STATISTICS);
+    store.withDurationTracking(DURATION_STATISTICS);
+    return store;
+  }
+
+  /**
+   * If the object is an IOStatisticsSource, get and add
+   * its IOStatistics.
+   * @param o source object.
+   */
+  public static void maybeAddIOStatistics(IOStatisticsAggregator ios,
+      Object o) {
+    if (o instanceof IOStatisticsSource) {
+      ios.aggregate(((IOStatisticsSource) o).getIOStatistics());
+    }
+  }
+
+  /**
+   * Build a Job UUID from the job conf (if it is
+   * {@link ManifestCommitterConstants#SPARK_WRITE_UUID}
+   * or the MR job ID.
+   * @param conf job/task configuration
+   * @param jobId job ID from YARN or spark.
+   * @return (a job ID, source)
+   */
+  public static Pair<String, String> buildJobUUID(Configuration conf,
+      JobID jobId) {
+    String jobUUID = conf.getTrimmed(SPARK_WRITE_UUID, "");

Review comment:
       long story.  see
   * [HADOOP-17318](https://issues.apache.org/jira/browse/HADOOP-17318) S3A 
committer to support concurrent jobs with same app attempt ID & dest dir
   * [SPARK-33402](https://issues.apache.org/jira/browse/SPARK-33402) Jobs 
launched in same second have duplicate MapReduce JobIDs
   * [SPARK-33230](https://issues.apache.org/jira/browse/SPARK-33230) 
FileOutputWriter jobs have duplicate JobIDs if launched in same second
   essentially: spark was launching jobs whose jobid was based on wall time, if 
two jobs started same second then things would collide. moving to using the 
uuid they (now) set everywhere guarantees this problem will not reoccur.




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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to