Skip to content

Commit e446287

Browse files
author
Ilya Ganelin
committed
Merge remote-tracking branch 'upstream/master' into SPARK-3288
2 parents b8c05cb + 1881431 commit e446287

File tree

229 files changed

+2722
-2853
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

229 files changed

+2722
-2853
lines changed

core/src/main/java/org/apache/spark/TaskContext.java

Lines changed: 22 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,7 @@ static void unset() {
6262
*/
6363
public abstract boolean isInterrupted();
6464

65-
/** @deprecated: use isRunningLocally() */
65+
/** @deprecated use {@link #isRunningLocally()} */
6666
@Deprecated
6767
public abstract boolean runningLocally();
6868

@@ -87,19 +87,39 @@ static void unset() {
8787
* is for HadoopRDD to register a callback to close the input stream.
8888
* Will be called in any situation - success, failure, or cancellation.
8989
*
90-
* @deprecated: use addTaskCompletionListener
90+
* @deprecated use {@link #addTaskCompletionListener(scala.Function1)}
9191
*
9292
* @param f Callback function.
9393
*/
9494
@Deprecated
9595
public abstract void addOnCompleteCallback(final Function0<Unit> f);
9696

97+
/**
98+
* The ID of the stage that this task belong to.
99+
*/
97100
public abstract int stageId();
98101

102+
/**
103+
* The ID of the RDD partition that is computed by this task.
104+
*/
99105
public abstract int partitionId();
100106

107+
/**
108+
* How many times this task has been attempted. The first task attempt will be assigned
109+
* attemptNumber = 0, and subsequent attempts will have increasing attempt numbers.
110+
*/
111+
public abstract int attemptNumber();
112+
113+
/** @deprecated use {@link #taskAttemptId()}; it was renamed to avoid ambiguity. */
114+
@Deprecated
101115
public abstract long attemptId();
102116

117+
/**
118+
* An ID that is unique to this task attempt (within the same SparkContext, no two task attempts
119+
* will share the same attempt ID). This is roughly equivalent to Hadoop's TaskAttemptID.
120+
*/
121+
public abstract long taskAttemptId();
122+
103123
/** ::DeveloperApi:: */
104124
@DeveloperApi
105125
public abstract TaskMetrics taskMetrics();

core/src/main/scala/org/apache/spark/TaskContextImpl.scala

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,14 +22,19 @@ import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerExce
2222

2323
import scala.collection.mutable.ArrayBuffer
2424

25-
private[spark] class TaskContextImpl(val stageId: Int,
25+
private[spark] class TaskContextImpl(
26+
val stageId: Int,
2627
val partitionId: Int,
27-
val attemptId: Long,
28+
override val taskAttemptId: Long,
29+
override val attemptNumber: Int,
2830
val runningLocally: Boolean = false,
2931
val taskMetrics: TaskMetrics = TaskMetrics.empty)
3032
extends TaskContext
3133
with Logging {
3234

35+
// For backwards-compatibility; this method is now deprecated as of 1.3.0.
36+
override def attemptId: Long = taskAttemptId
37+
3338
// List of callback functions to execute when the task completes.
3439
@transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener]
3540

core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -149,6 +149,11 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St
149149
// Global defaults. These should be keep to minimum to avoid confusing behavior.
150150
master = Option(master).getOrElse("local[*]")
151151

152+
// In YARN mode, app name can be set via SPARK_YARN_APP_NAME (see SPARK-5222)
153+
if (master.startsWith("yarn")) {
154+
name = Option(name).orElse(env.get("SPARK_YARN_APP_NAME")).orNull
155+
}
156+
152157
// Set name from main class if not given
153158
name = Option(name).orElse(Option(mainClass)).orNull
154159
if (name == null && primaryResource != null) {

core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,8 @@ private[spark] class CoarseGrainedExecutorBackend(
7171
val ser = env.closureSerializer.newInstance()
7272
val taskDesc = ser.deserialize[TaskDescription](data.value)
7373
logInfo("Got assigned task " + taskDesc.taskId)
74-
executor.launchTask(this, taskDesc.taskId, taskDesc.name, taskDesc.serializedTask)
74+
executor.launchTask(this, taskId = taskDesc.taskId, attemptNumber = taskDesc.attemptNumber,
75+
taskDesc.name, taskDesc.serializedTask)
7576
}
7677

7778
case KillTask(taskId, _, interruptThread) =>

core/src/main/scala/org/apache/spark/executor/Executor.scala

Lines changed: 13 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -108,8 +108,13 @@ private[spark] class Executor(
108108
startDriverHeartbeater()
109109

110110
def launchTask(
111-
context: ExecutorBackend, taskId: Long, taskName: String, serializedTask: ByteBuffer) {
112-
val tr = new TaskRunner(context, taskId, taskName, serializedTask)
111+
context: ExecutorBackend,
112+
taskId: Long,
113+
attemptNumber: Int,
114+
taskName: String,
115+
serializedTask: ByteBuffer) {
116+
val tr = new TaskRunner(context, taskId = taskId, attemptNumber = attemptNumber, taskName,
117+
serializedTask)
113118
runningTasks.put(taskId, tr)
114119
threadPool.execute(tr)
115120
}
@@ -134,7 +139,11 @@ private[spark] class Executor(
134139
private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum
135140

136141
class TaskRunner(
137-
execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer)
142+
execBackend: ExecutorBackend,
143+
val taskId: Long,
144+
val attemptNumber: Int,
145+
taskName: String,
146+
serializedTask: ByteBuffer)
138147
extends Runnable {
139148

140149
@volatile private var killed = false
@@ -180,7 +189,7 @@ private[spark] class Executor(
180189

181190
// Run the actual task and measure its runtime.
182191
taskStart = System.currentTimeMillis()
183-
val value = task.run(taskId.toInt)
192+
val value = task.run(taskAttemptId = taskId, attemptNumber = attemptNumber)
184193
val taskFinish = System.currentTimeMillis()
185194

186195
// If the task has been killed, let's fail it.

core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _}
2828
import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv}
2929
import org.apache.spark.TaskState.TaskState
3030
import org.apache.spark.deploy.SparkHadoopUtil
31+
import org.apache.spark.scheduler.cluster.mesos.{MesosTaskLaunchData}
3132
import org.apache.spark.util.{SignalLogger, Utils}
3233

3334
private[spark] class MesosExecutorBackend
@@ -77,11 +78,13 @@ private[spark] class MesosExecutorBackend
7778

7879
override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) {
7980
val taskId = taskInfo.getTaskId.getValue.toLong
81+
val taskData = MesosTaskLaunchData.fromByteString(taskInfo.getData)
8082
if (executor == null) {
8183
logError("Received launchTask but executor was null")
8284
} else {
8385
SparkHadoopUtil.get.runAsSparkUser { () =>
84-
executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer)
86+
executor.launchTask(this, taskId = taskId, attemptNumber = taskData.attemptNumber,
87+
taskInfo.getName, taskData.serializedTask)
8588
}
8689
}
8790
}

core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -95,7 +95,8 @@ private[spark] object CheckpointRDD extends Logging {
9595

9696
val finalOutputName = splitIdToFile(ctx.partitionId)
9797
val finalOutputPath = new Path(outputDir, finalOutputName)
98-
val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId)
98+
val tempOutputPath =
99+
new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptNumber)
99100

100101
if (fs.exists(tempOutputPath)) {
101102
throw new IOException("Checkpoint failed: temporary path " +
@@ -119,7 +120,7 @@ private[spark] object CheckpointRDD extends Logging {
119120
logInfo("Deleting tempOutputPath " + tempOutputPath)
120121
fs.delete(tempOutputPath, false)
121122
throw new IOException("Checkpoint failed: failed to save output of task: "
122-
+ ctx.attemptId + " and final output path does not exist")
123+
+ ctx.attemptNumber + " and final output path does not exist")
123124
} else {
124125
// Some other copy of this task must've finished before us and renamed it
125126
logInfo("Final output path " + finalOutputPath + " already exists; not overwriting it")

core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -229,7 +229,7 @@ class HadoopRDD[K, V](
229229
var reader: RecordReader[K, V] = null
230230
val inputFormat = getInputFormat(jobConf)
231231
HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime),
232-
context.stageId, theSplit.index, context.attemptId.toInt, jobConf)
232+
context.stageId, theSplit.index, context.attemptNumber, jobConf)
233233
reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
234234

235235
// Register an on-task-completion callback to close the input stream.

core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -978,12 +978,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
978978

979979
val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => {
980980
val config = wrappedConf.value
981-
// Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
982-
// around by taking a mod. We expect that no task will be attempted 2 billion times.
983-
val attemptNumber = (context.attemptId % Int.MaxValue).toInt
984981
/* "reduce task" <split #> <attempt # = spark task #> */
985982
val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId,
986-
attemptNumber)
983+
context.attemptNumber)
987984
val hadoopContext = newTaskAttemptContext(config, attemptId)
988985
val format = outfmt.newInstance
989986
format match {
@@ -1062,11 +1059,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
10621059
val config = wrappedConf.value
10631060
// Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
10641061
// around by taking a mod. We expect that no task will be attempted 2 billion times.
1065-
val attemptNumber = (context.attemptId % Int.MaxValue).toInt
1062+
val taskAttemptId = (context.taskAttemptId % Int.MaxValue).toInt
10661063

10671064
val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context, config)
10681065

1069-
writer.setup(context.stageId, context.partitionId, attemptNumber)
1066+
writer.setup(context.stageId, context.partitionId, taskAttemptId)
10701067
writer.open()
10711068
try {
10721069
var recordsWritten = 0L

core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -635,8 +635,8 @@ class DAGScheduler(
635635
try {
636636
val rdd = job.finalStage.rdd
637637
val split = rdd.partitions(job.partitions(0))
638-
val taskContext =
639-
new TaskContextImpl(job.finalStage.id, job.partitions(0), 0, true)
638+
val taskContext = new TaskContextImpl(job.finalStage.id, job.partitions(0), taskAttemptId = 0,
639+
attemptNumber = 0, runningLocally = true)
640640
TaskContextHelper.setTaskContext(taskContext)
641641
try {
642642
val result = job.func(taskContext, rdd.iterator(split, taskContext))

0 commit comments

Comments
 (0)