Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-21549][CORE] Respect OutputFormats with no output directory provided #19294

Closed
wants to merge 15 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
15 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,9 @@ import org.apache.spark.mapred.SparkHadoopMapRedUtil
* (from the newer mapreduce API, not the old mapred API).
*
* Unlike Hadoop's OutputCommitter, this implementation is serializable.
*
* @param jobId the job's or stage's id
* @param path the job's output path, or null if committer acts as a noop
*/
class HadoopMapReduceCommitProtocol(jobId: String, path: String)
extends FileCommitProtocol with Serializable with Logging {
Expand All @@ -57,6 +60,15 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
*/
private def absPathStagingDir: Path = new Path(path, "_temporary-" + jobId)

/**
* Checks whether there are files to be committed to an absolute output location.
*
* As committing and aborting a job occurs on driver, where `addedAbsPathFiles` is always null,
* it is necessary to check whether the output path is specified. Output path may not be required
* for committers not writing to distributed file systems.
*/
private def hasAbsPathFiles: Boolean = path != null

protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = {
val format = context.getOutputFormatClass.newInstance()
// If OutputFormat is Configurable, we should set conf to it.
Expand Down Expand Up @@ -130,17 +142,21 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]])
.foldLeft(Map[String, String]())(_ ++ _)
logDebug(s"Committing files staged for absolute locations $filesToMove")
val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
for ((src, dst) <- filesToMove) {
fs.rename(new Path(src), new Path(dst))
if (hasAbsPathFiles) {
val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
for ((src, dst) <- filesToMove) {
fs.rename(new Path(src), new Path(dst))
}
fs.delete(absPathStagingDir, true)
}
fs.delete(absPathStagingDir, true)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Given the changes being made here, it seems a good place to add the suggestion of SPARK-20045 & make that abort() call resilient to failures, by doing that delete even if the hadoop committer raised an IOE

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wouldn't it be better to fix it in separate PR?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can do, now you've got a little mock committer in someone can just extend it to optionally throw an IOE in abort().

}

override def abortJob(jobContext: JobContext): Unit = {
committer.abortJob(jobContext, JobStatus.State.FAILED)
val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
fs.delete(absPathStagingDir, true)
if (hasAbsPathFiles) {
val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
fs.delete(absPathStagingDir, true)
}
}

override def setupTask(taskContext: TaskAttemptContext): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ import org.apache.commons.math3.distribution.{BinomialDistribution, PoissonDistr
import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.mapred._
import org.apache.hadoop.mapreduce.{JobContext => NewJobContext,
import org.apache.hadoop.mapreduce.{Job => NewJob, JobContext => NewJobContext,
OutputCommitter => NewOutputCommitter, OutputFormat => NewOutputFormat,
RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext}
import org.apache.hadoop.util.Progressable
Expand Down Expand Up @@ -568,6 +568,37 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
assert(FakeWriterWithCallback.exception.getMessage contains "failed to write")
}

test("saveAsNewAPIHadoopDataset should respect empty output directory when " +
"there are no files to be committed to an absolute output location") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)

val job = NewJob.getInstance(new Configuration(sc.hadoopConfiguration))
job.setOutputKeyClass(classOf[Integer])
job.setOutputValueClass(classOf[Integer])
job.setOutputFormatClass(classOf[NewFakeFormat])
val jobConfiguration = job.getConfiguration

// just test that the job does not fail with
// java.lang.IllegalArgumentException: Can not create a Path from a null string
pairs.saveAsNewAPIHadoopDataset(jobConfiguration)
}

test("saveAsHadoopDataset should respect empty output directory when " +
"there are no files to be committed to an absolute output location") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)

val conf = new JobConf()
conf.setOutputKeyClass(classOf[Integer])
conf.setOutputValueClass(classOf[Integer])
conf.setOutputFormat(classOf[FakeOutputFormat])
conf.setOutputCommitter(classOf[FakeOutputCommitter])

FakeOutputCommitter.ran = false
pairs.saveAsHadoopDataset(conf)

assert(FakeOutputCommitter.ran, "OutputCommitter was never called")
}

test("lookup") {
val pairs = sc.parallelize(Array((1, 2), (3, 4), (5, 6), (5, 7)))

Expand Down