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-6369] [SQL] Uses commit coordinator to help committing Hive and Parquet tables #5139
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,9 +17,15 @@ | |
|
||
package org.apache.spark.mapred | ||
|
||
import java.io.IOException | ||
import java.lang.reflect.Modifier | ||
|
||
import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} | ||
import org.apache.hadoop.mapred._ | ||
import org.apache.hadoop.mapreduce.{TaskAttemptContext => MapReduceTaskAttemptContext} | ||
import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter} | ||
|
||
import org.apache.spark.executor.CommitDeniedException | ||
import org.apache.spark.{Logging, SparkEnv, TaskContext} | ||
|
||
private[spark] | ||
trait SparkHadoopMapRedUtil { | ||
|
@@ -65,3 +71,77 @@ trait SparkHadoopMapRedUtil { | |
} | ||
} | ||
} | ||
|
||
object SparkHadoopMapRedUtil extends Logging { | ||
def commitTask( | ||
committer: MapReduceOutputCommitter, | ||
mrTaskContext: MapReduceTaskAttemptContext, | ||
sparkTaskContext: TaskContext): Unit = { | ||
commitTask( | ||
committer, | ||
mrTaskContext, | ||
sparkTaskContext.stageId(), | ||
sparkTaskContext.partitionId(), | ||
sparkTaskContext.attemptNumber()) | ||
} | ||
|
||
def commitTask( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add documentation to this guy mentioning what it means to commitTask (i.e., we may contact the driver to become authorized to commit to ensure speculative tasks do not override each other, and that this may cause us to abort the task by throwing a CommitDeniedException if we cannot become authorized as such), pointing to the JIRA that this fixes (the original one). |
||
committer: MapReduceOutputCommitter, | ||
mrTaskContext: MapReduceTaskAttemptContext, | ||
jobId: Int, | ||
splitId: Int, | ||
attemptId: Int): Unit = { | ||
|
||
val mrTaskAttemptID = mrTaskContext.getTaskAttemptID | ||
|
||
// Called after we have decided to commit | ||
def performCommit(): Unit = { | ||
try { | ||
committer.commitTask(mrTaskContext) | ||
logInfo(s"$mrTaskAttemptID: Committed") | ||
} catch { | ||
case cause: IOException => | ||
logError(s"Error committing the output of task: $mrTaskAttemptID", cause) | ||
committer.abortTask(mrTaskContext) | ||
throw cause | ||
} | ||
} | ||
|
||
// First, check whether the task's output has already been committed by some other attempt | ||
if (committer.needsTaskCommit(mrTaskContext)) { | ||
// The task output needs to be committed, but we don't know whether some other task attempt | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In fact, it may be sufficient to lift and modify this comment up to the javadoc. |
||
// might be racing to commit the same output partition. Therefore, coordinate with the driver | ||
// in order to determine whether this attempt can commit (see SPARK-4879). | ||
val shouldCoordinateWithDriver: Boolean = { | ||
val sparkConf = SparkEnv.get.conf | ||
// We only need to coordinate with the driver if there are multiple concurrent task | ||
// attempts, which should only occur if speculation is enabled | ||
val speculationEnabled = sparkConf.getBoolean("spark.speculation", defaultValue = false) | ||
// This (undocumented) setting is an escape-hatch in case the commit code introduces bugs | ||
sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", speculationEnabled) | ||
} | ||
|
||
if (shouldCoordinateWithDriver) { | ||
val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator | ||
val canCommit = outputCommitCoordinator.canCommit(jobId, splitId, attemptId) | ||
|
||
if (canCommit) { | ||
performCommit() | ||
} else { | ||
val message = | ||
s"$mrTaskAttemptID: Not committed because the driver did not authorize commit" | ||
logInfo(message) | ||
// We need to abort the task so that the driver can reschedule new attempts, if necessary | ||
committer.abortTask(mrTaskContext) | ||
throw new CommitDeniedException(message, jobId, splitId, attemptId) | ||
} | ||
} else { | ||
// Speculation is disabled or a user has chosen to manually bypass the commit coordination | ||
performCommit() | ||
} | ||
} else { | ||
// Some other attempt committed the output, so we do nothing and signal success | ||
logInfo(s"No need to commit output of task because needsTaskCommit=false: $mrTaskAttemptID") | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: maybe place this overloaded version below the main one as it is merely a convenience method (also allows us to avoid duplicating the documentation)