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_r309641936
########## 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) + } + + logDebug(s"Using committer ${committer.getClass}") + logDebug(s"Committer details: $committer") + if (committer.isInstanceOf[FileOutputCommitter]) { + require(!rejectFileOutput, + s"Committer created is the FileOutputCommitter $committer") + + if (committer.isCommitJobRepeatable(context)) { + // If FileOutputCommitter says its job commit is repeatable, it means + // it is using the v2 algorithm, which is not safe for task commit + // failures. Warn + logDebug(s"Committer $committer may not be tolerant of task commit failures") + } + } + committer + } + + /** + * Create a temporary file for a task. + * + * @param taskContext task context + * @param dir optional subdirectory + * @param ext file extension + * @return a path as a string + */ + override def newTaskTempFile( + taskContext: TaskAttemptContext, + dir: Option[String], + ext: String): String = { + + val workDir = committer.getWorkPath + val parent = dir.map { + d => new Path(workDir, d) + }.getOrElse(workDir) + val file = new Path(parent, buildFilename(taskContext, ext)) + logDebug(s"Creating task file $file for dir $dir and ext $ext") + file.toString + } + + /** + * Request a path for new output file to be committed in job + * commit into the absolute path specified. + * + * The created file will be committed through the `rename()` operation + * of the filesystem/filestore connector, which may be a slow `O(data)` Review comment: this is only for those cases where the protocol wants an absolute path. Traditionally spark cast the committer to a FileOutputCommitter and used methods there. Rather than make changes to what is a scarily complex piece of critical code to support new plugin committers, I just pulled up the relevant method into a new interface which can be used instead. Normally work goes into the subdirectory of the output, and even when partitioned there all is well. The PartitionedCommitter Ryan Blue wrote even does conflict handling within the partitions, so you can run a job targeting an existing output directory, and have it address conflict not at the base "fail if dest exists", but act on the final leaves of the partition with "rm * in that final dir", "fail if that final dir isn't empty" or "just add things relying on UUID insertion to avoid conflict and S3 update inconsistencies" Happy to explore this more in future. FWIW, I think spark should be able to query a committer to see if could recover from a failure/partition during commit and then fail a job if a committer "went away" during that process. The existing MRv2 committer would return false here, as should the new EMR one. MRv1 and these S3A committers are resilient to failure here ... which is exactly what spark and Hadoop MR actually assume. ---------------------------------------------------------------- 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] With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
