Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -698,7 +698,7 @@ private[spark] class DAGScheduler(
if (partitions.isEmpty) {
val time = clock.getTimeMillis()
listenerBus.post(
SparkListenerJobStart(jobId, time, Seq[StageInfo](), properties))
SparkListenerJobStart(jobId, time, Seq[StageInfo](), SerializationUtils.clone(properties)))
listenerBus.post(
SparkListenerJobEnd(jobId, time, JobSucceeded))
// Return immediately if the job is running 0 tasks
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,14 @@ package org.apache.spark.scheduler

import java.util.Properties
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicLong, AtomicReference}

import scala.annotation.meta.param
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
import scala.util.control.NonFatal

import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits}
import org.scalatest.exceptions.TestFailedException
import org.scalatest.time.SpanSugar._

import org.apache.spark._
Expand All @@ -36,7 +37,7 @@ import org.apache.spark.rdd.{DeterministicLevel, RDD}
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException}
import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils}
import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, ThreadUtils, Utils}

class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler)
extends DAGSchedulerEventProcessLoop(dagScheduler) {
Expand Down Expand Up @@ -788,6 +789,36 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi
}
}

test("SPARK-28967 properties must be cloned before posting to listener bus for 0 partition") {
val properties = new Properties()
val func = (context: TaskContext, it: Iterator[(_)]) => 1
val resultHandler = (taskIndex: Int, result: Int) => {}
val assertionError = new AtomicReference[TestFailedException](
new TestFailedException("Listener didn't receive expected JobStart event", 0))
val listener = new SparkListener() {
override def onJobStart(event: SparkListenerJobStart): Unit = {
try {
assert(event.properties.equals(properties), "Expected same content of properties, " +
s"but got properties with different content. props in caller ${properties} /" +
s" props in event ${event.properties}")
assert(event.properties.ne(properties), "Expected instance with different identity, " +
"but got same instance.")
assertionError.set(null)
} catch {
case e: TestFailedException => assertionError.set(e)
}
}
}
sc.addSparkListener(listener)

// 0 partition
val testRdd = new MyRDD(sc, 0, Nil)
val waiter = scheduler.submitJob(testRdd, func, Seq.empty, CallSite.empty,
resultHandler, properties)
sc.listenerBus.waitUntilEmpty(1000L)
assert(assertionError.get() === null)
}

// Helper function to validate state when creating tests for task failures
private def checkStageId(stageId: Int, attempt: Int, stageAttempt: TaskSet) {
assert(stageAttempt.stageId === stageId)
Expand Down