You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/07/31 14:35:28 UTC

[GitHub] [spark] steveloughran commented on a change in pull request #24970: [SPARK-23977][SQL] Support High Performance S3A committers [test-hadoop3.2]

steveloughran commented on a change in pull request #24970: [SPARK-23977][SQL] Support High Performance S3A committers [test-hadoop3.2]
URL: https://github.com/apache/spark/pull/24970#discussion_r309256164
 
 

 ##########
 File path: hadoop-cloud/src/hadoop-3/main/scala/org/apache/spark/internal/io/cloud/PathOutputCommitProtocol.scala
 ##########
 @@ -0,0 +1,263 @@
+/*
+ * 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.spark.internal.io.cloud
+
+import java.io.IOException
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext}
+import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, PathOutputCommitter, PathOutputCommitterFactory}
+
+import org.apache.spark.internal.io.{FileCommitProtocol, HadoopMapReduceCommitProtocol}
+
+/**
+ * Spark Commit protocol for Path Output Committers.
+ * This committer will work with the `FileOutputCommitter` and subclasses.
+ * All implementations *must* be serializable.
+ *
+ * Rather than ask the `FileOutputFormat` for a committer, it uses the
+ * `org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory` factory
+ * API to create the committer.
+ *
+ * In `setupCommitter` the factory is identified and instantiated to;
+ * this factory then creates the actual committer implementation.
+ *
+ * @constructor Instantiate. dynamic partition overwrite is not supported,
+ *              so that committers for stores which do not support rename
+ *              will not get confused.
+ * @param jobId                     job
+ * @param dest                      destination
+ * @param dynamicPartitionOverwrite does the caller want support for dynamic
+ *                                  partition overwrite. If so, it will be
+ *                                  refused.
+ * @throws IOException when an unsupported dynamicPartitionOverwrite option is supplied.
+ */
+class PathOutputCommitProtocol(
+    jobId: String,
+    dest: String,
+    dynamicPartitionOverwrite: Boolean = false)
+  extends HadoopMapReduceCommitProtocol(jobId, dest, false) with Serializable {
+
+  if (dynamicPartitionOverwrite) {
+    // until there's explicit extensions to the PathOutputCommitProtocols
+    // to support the spark mechanism, it's left to the individual committer
+    // choice to handle partitioning.
+    throw new IOException(PathOutputCommitProtocol.UNSUPPORTED)
+  }
+
+  /** The committer created. */
+  @transient private var committer: PathOutputCommitter = _
+
+  require(dest != null, "Null destination specified")
+
+  private[cloud] val destination: String = dest
+
+  /** The destination path. This is serializable in Hadoop 3. */
+  private[cloud] val destPath: Path = new Path(destination)
+
+  logDebug(s"Instantiated committer with job ID=$jobId;" +
+    s" destination=$destPath;" +
+    s" dynamicPartitionOverwrite=$dynamicPartitionOverwrite")
+
+  import PathOutputCommitProtocol._
+
+  /**
+   * Set up the committer.
+   * This creates it by talking directly to the Hadoop factories, instead
+   * of the V1 `mapred.FileOutputFormat` methods.
+   * @param context task attempt
+   * @return the committer to use. This will always be a subclass of
+   *         [[PathOutputCommitter]].
+   */
+  override protected def setupCommitter(context: TaskAttemptContext): PathOutputCommitter = {
+
+    logDebug(s"Setting up committer for path $destination")
+    committer = PathOutputCommitterFactory.createCommitter(destPath, context)
+
+    // Special feature to force out the FileOutputCommitter, so as to guarantee
+    // that the binding is working properly.
+    val rejectFileOutput = context.getConfiguration
+      .getBoolean(REJECT_FILE_OUTPUT, REJECT_FILE_OUTPUT_DEFVAL)
+    if (rejectFileOutput && committer.isInstanceOf[FileOutputCommitter]) {
+      // the output format returned a file output format committer, which
+      // is exactly what we do not want. So switch back to the factory.
+      val factory = PathOutputCommitterFactory.getCommitterFactory(
+        destPath,
+        context.getConfiguration)
+      logDebug(s"Using committer factory $factory")
+      committer = factory.createOutputCommitter(destPath, context)
 
 Review comment:
   It's more sometimes you want the option to say "really, really reject the classic"; a guard against that default actually happening.
   
   The MR committer factories get to choose their own committers; currently the S3A one defaults to (still) using the rename one as switching over to a newer one. I think with HADOOP-16357 we've got the staging committer consistent with what MR expects w.r.t whether or not the destination directory is empty on startup& so for MR and spark work you should be able to switch over to it without worries. A little option here lets you fail fast on job launch to make sure that your job really has done that.
   
   without that, you only get to find out that you are still using it in the good case (it takes a lot longer and you find that `_SUCCESS` is a 0-byte file), or the bad case (it takes a lot longer, you find that `_SUCCESS` is a 0-byte file and because of failures during task commit, your output has been silently corrupted) (*)
   
   (*) I am of the believe that failure condition holds for FileOutputCommitter v2 and any store, it's just that for S3 and other slow-to-rename stores the window of vulnerability is a lot longer, and, without heartbeats coming back, there's a risk of the rename actually triggering a timeout. Never heard of that with Spark, but it has been known to happen with Distcp of a many GB file

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


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org