Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
37 commits
Select commit Hold shift + click to select a range
999ec13
[SPARK-22570][SQL] Avoid to create a lot of global variables by using…
kiszk Nov 30, 2017
6ac57fd
[SPARK-21417][SQL] Infer join conditions using propagated constraints
Nov 30, 2017
bcceab6
[SPARK-22489][SQL] Shouldn't change broadcast join buildSide if user …
wangyum Nov 30, 2017
f5f8e84
[SPARK-22614] Dataset API: repartitionByRange(...)
adrian-ionescu Nov 30, 2017
7e5f669
[SPARK-22428][DOC] Add spark application garbage collector configurat…
gaborgsomogyi Dec 1, 2017
7da1f57
[SPARK-22373] Bump Janino dependency version to fix thread safety issue…
Victsm Dec 1, 2017
dc36542
[SPARK-22653] executorAddress registered in CoarseGrainedSchedulerBac…
tgravescs Dec 1, 2017
16adaf6
[SPARK-22601][SQL] Data load is getting displayed successful on provi…
sujith71955 Dec 1, 2017
9d06a9e
[SPARK-22393][SPARK-SHELL] spark-shell can't find imported types in c…
mpetruska Dec 1, 2017
ee10ca7
[SPARK-22638][SS] Use a separate queue for StreamingQueryListenerBus
zsxwing Dec 1, 2017
aa4cf2b
[SPARK-22651][PYTHON][ML] Prevent initiating multiple Hive clients fo…
HyukjinKwon Dec 2, 2017
d2cf95a
[SPARK-22634][BUILD] Update Bouncy Castle to 1.58
srowen Dec 2, 2017
f23dddf
[SPARK-20682][SPARK-15474][SPARK-21791] Add new ORCFileFormat based o…
dongjoon-hyun Dec 3, 2017
2c16267
[SPARK-22669][SQL] Avoid unnecessary function calls in code generation
mgaido91 Dec 3, 2017
dff440f
[SPARK-22626][SQL] deals with wrong Hive's statistics (zero rowCount)
wangyum Dec 3, 2017
4131ad0
[SPARK-22489][DOC][FOLLOWUP] Update broadcast behavior changes in mig…
wangyum Dec 4, 2017
3927bb9
[SPARK-22473][FOLLOWUP][TEST] Remove deprecated Date functions
mgaido91 Dec 4, 2017
f81401e
[SPARK-22162] Executors and the driver should use consistent JobIDs i…
Dec 4, 2017
e1dd03e
[SPARK-22372][CORE, YARN] Make cluster submission use SparkApplication.
Dec 4, 2017
dcaac45
Spark on Kubernetes - basic submission client
liyinan926 Nov 10, 2017
27c67ff
Addressed first round of review comments
liyinan926 Nov 27, 2017
6d597d0
Made Client implement the SparkApplication trait
liyinan926 Nov 28, 2017
5b9fa39
Addressed the second round of comments
liyinan926 Nov 28, 2017
5ccadb5
Added missing step for supporting local:// dependencies and addressed…
liyinan926 Nov 30, 2017
12f2797
Fixed Scala style check errors
liyinan926 Nov 30, 2017
c35fe48
Addressed another round of comments
liyinan926 Dec 4, 2017
faa2849
Rebased on master and added a constant val for the Client class
liyinan926 Dec 4, 2017
347ed69
Addressed another major round of comments
liyinan926 Dec 5, 2017
0e8ca01
Addressed one more round of comments
liyinan926 Dec 5, 2017
3a0b8e3
Removed mentioning of kubernetes-namespace
liyinan926 Dec 6, 2017
83d0b9c
Fixed a couple of bugs found during manual tests
liyinan926 Dec 7, 2017
44c40b1
Guard against client mode in SparkContext
liyinan926 Dec 8, 2017
67bc847
Added libc6-compat into the base docker image
liyinan926 Dec 8, 2017
7d2b303
Addressed latest comments
liyinan926 Dec 8, 2017
caf2206
Addressed docs comments
liyinan926 Dec 9, 2017
2e7810b
Fixed a comment
liyinan926 Dec 11, 2017
cbcd30e
Addressed latest comments
liyinan926 Dec 11, 2017
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
Prev Previous commit
Next Next commit
[SPARK-22162] Executors and the driver should use consistent JobIDs i…
…n the RDD commit protocol

I have modified SparkHadoopWriter so that executors and the driver always use consistent JobIds during the hadoop commit. Before SPARK-18191, spark always used the rddId, it just incorrectly named the variable stageId. After SPARK-18191, it used the rddId as the jobId on the driver's side, and the stageId as the jobId on the executors' side. With this change executors and the driver will consistently uses rddId as the jobId. Also with this change, during the hadoop commit protocol spark uses  actual stageId to check whether a stage can be committed unlike before that  it was using executors' jobId to do this check.
In addition to the existing unit tests, a test has been added to check whether executors and the driver are using the same JobId. The test failed before this change and passed after applying this fix.

Author: Reza Safi <[email protected]>

Closes apache#19848 from rezasafi/stagerddsimple.
  • Loading branch information
Reza Safi authored and Marcelo Vanzin committed Dec 4, 2017
commit f81401e1cb39f2d6049b79dc8d61305f3371276f
Original file line number Diff line number Diff line change
Expand Up @@ -60,17 +60,17 @@ object SparkHadoopWriter extends Logging {
config: HadoopWriteConfigUtil[K, V]): Unit = {
// Extract context and configuration from RDD.
val sparkContext = rdd.context
val stageId = rdd.id
val commitJobId = rdd.id

// Set up a job.
val jobTrackerId = createJobTrackerID(new Date())
val jobContext = config.createJobContext(jobTrackerId, stageId)
val jobContext = config.createJobContext(jobTrackerId, commitJobId)
config.initOutputFormat(jobContext)

// Assert the output format/key/value class is set in JobConf.
config.assertConf(jobContext, rdd.conf)

val committer = config.createCommitter(stageId)
val committer = config.createCommitter(commitJobId)
committer.setupJob(jobContext)

// Try to write all RDD partitions as a Hadoop OutputFormat.
Expand All @@ -80,7 +80,7 @@ object SparkHadoopWriter extends Logging {
context = context,
config = config,
jobTrackerId = jobTrackerId,
sparkStageId = context.stageId,
commitJobId = commitJobId,
sparkPartitionId = context.partitionId,
sparkAttemptNumber = context.attemptNumber,
committer = committer,
Expand All @@ -102,14 +102,14 @@ object SparkHadoopWriter extends Logging {
context: TaskContext,
config: HadoopWriteConfigUtil[K, V],
jobTrackerId: String,
sparkStageId: Int,
commitJobId: Int,
sparkPartitionId: Int,
sparkAttemptNumber: Int,
committer: FileCommitProtocol,
iterator: Iterator[(K, V)]): TaskCommitMessage = {
// Set up a task.
val taskContext = config.createTaskAttemptContext(
jobTrackerId, sparkStageId, sparkPartitionId, sparkAttemptNumber)
jobTrackerId, commitJobId, sparkPartitionId, sparkAttemptNumber)
committer.setupTask(taskContext)

val (outputMetrics, callback) = initHadoopOutputMetrics(context)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,8 @@ object SparkHadoopMapRedUtil extends Logging {
if (shouldCoordinateWithDriver) {
val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator
val taskAttemptNumber = TaskContext.get().attemptNumber()
val canCommit = outputCommitCoordinator.canCommit(jobId, splitId, taskAttemptNumber)
val stageId = TaskContext.get().stageId()
val canCommit = outputCommitCoordinator.canCommit(stageId, splitId, taskAttemptNumber)

if (canCommit) {
performCommit()
Expand All @@ -80,7 +81,7 @@ object SparkHadoopMapRedUtil extends Logging {
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, taskAttemptNumber)
throw new CommitDeniedException(message, stageId, splitId, taskAttemptNumber)
}
} else {
// Speculation is disabled or a user has chosen to manually bypass the commit coordination
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import org.apache.hadoop.mapreduce.{Job => NewJob, JobContext => NewJobContext,
OutputCommitter => NewOutputCommitter, OutputFormat => NewOutputFormat,
RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext}
import org.apache.hadoop.util.Progressable
import org.scalatest.Assertions

import org.apache.spark._
import org.apache.spark.Partitioner
Expand Down Expand Up @@ -524,6 +525,15 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored")
}

test("The JobId on the driver and executors should be the same during the commit") {
// Create more than one rdd to mimic stageId not equal to rddId
val pairs = sc.parallelize(Array((1, 2), (2, 3)), 2)
.map { p => (new Integer(p._1 + 1), new Integer(p._2 + 1)) }
.filter { p => p._1 > 0 }
pairs.saveAsNewAPIHadoopFile[YetAnotherFakeFormat]("ignored")
assert(JobID.jobid != -1)
}

test("saveAsHadoopFile should respect configured output committers") {
val pairs = sc.parallelize(Array((new Integer(1), new Integer(1))))
val conf = new JobConf()
Expand Down Expand Up @@ -908,6 +918,40 @@ class NewFakeFormatWithCallback() extends NewFakeFormat {
}
}

class YetAnotherFakeCommitter extends NewOutputCommitter with Assertions {
def setupJob(j: NewJobContext): Unit = {
JobID.jobid = j.getJobID().getId
}

def needsTaskCommit(t: NewTaskAttempContext): Boolean = false

def setupTask(t: NewTaskAttempContext): Unit = {
val jobId = t.getTaskAttemptID().getJobID().getId
assert(jobId === JobID.jobid)
}

def commitTask(t: NewTaskAttempContext): Unit = {}

def abortTask(t: NewTaskAttempContext): Unit = {}
}

class YetAnotherFakeFormat() extends NewOutputFormat[Integer, Integer]() {

def checkOutputSpecs(j: NewJobContext): Unit = {}

def getRecordWriter(t: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = {
new NewFakeWriter()
}

def getOutputCommitter(t: NewTaskAttempContext): NewOutputCommitter = {
new YetAnotherFakeCommitter()
}
}

object JobID {
var jobid = -1
}

class ConfigTestFormat() extends NewFakeFormat() with Configurable {

var setConfCalled = false
Expand Down