Skip to content

Commit

Permalink
[SPARK-23821][SQL] Merging current master to the feature branch
Browse files Browse the repository at this point in the history
  • Loading branch information
mn-mikke authored and mn-mikke committed Apr 10, 2018
2 parents e213341 + 6498884 commit b9d99f7
Show file tree
Hide file tree
Showing 57 changed files with 771 additions and 306 deletions.
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.expressions;

import org.apache.spark.unsafe.memory.MemoryBlock;
import org.apache.spark.unsafe.types.UTF8String;

/**
* Simulates Hive's hashing function from Hive v1.2.1
Expand Down Expand Up @@ -51,4 +52,8 @@ public static int hashUnsafeBytesBlock(MemoryBlock mb) {
public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) {
return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes));
}

public static int hashUTF8String(UTF8String str) {
return hashUnsafeBytesBlock(str.getMemoryBlock());
}
}
Expand Up @@ -20,6 +20,7 @@
import com.google.common.primitives.Ints;

import org.apache.spark.unsafe.memory.MemoryBlock;
import org.apache.spark.unsafe.types.UTF8String;

/**
* 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction.
Expand Down Expand Up @@ -82,6 +83,10 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) {
return fmix(h1, lengthInBytes);
}

public static int hashUTF8String(UTF8String str, int seed) {
return hashUnsafeBytesBlock(str.getMemoryBlock(), seed);
}

public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) {
return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed);
}
Expand All @@ -91,7 +96,7 @@ public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes,
}

public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) {
// This is compatible with original and another implementations.
// This is compatible with original and other implementations.
// Use this method for new components after Spark 2.3.
int lengthInBytes = Ints.checkedCast(base.size());
assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative";
Expand Down
26 changes: 13 additions & 13 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Expand Up @@ -480,6 +480,19 @@ private[spark] class Executor(
execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult)

} catch {
case t: TaskKilledException =>
logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}")
setTaskFinishedAndClearInterruptStatus()
execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled(t.reason)))

case _: InterruptedException | NonFatal(_) if
task != null && task.reasonIfKilled.isDefined =>
val killReason = task.reasonIfKilled.getOrElse("unknown reason")
logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason")
setTaskFinishedAndClearInterruptStatus()
execBackend.statusUpdate(
taskId, TaskState.KILLED, ser.serialize(TaskKilled(killReason)))

case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) =>
val reason = task.context.fetchFailed.get.toTaskFailedReason
if (!t.isInstanceOf[FetchFailedException]) {
Expand All @@ -494,19 +507,6 @@ private[spark] class Executor(
setTaskFinishedAndClearInterruptStatus()
execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))

case t: TaskKilledException =>
logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}")
setTaskFinishedAndClearInterruptStatus()
execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled(t.reason)))

case _: InterruptedException | NonFatal(_) if
task != null && task.reasonIfKilled.isDefined =>
val killReason = task.reasonIfKilled.getOrElse("unknown reason")
logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason")
setTaskFinishedAndClearInterruptStatus()
execBackend.statusUpdate(
taskId, TaskState.KILLED, ser.serialize(TaskKilled(killReason)))

case CausedBy(cDE: CommitDeniedException) =>
val reason = cDE.toTaskCommitDeniedReason
setTaskFinishedAndClearInterruptStatus()
Expand Down
31 changes: 18 additions & 13 deletions core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
Expand Up @@ -332,13 +332,15 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
import JobCancellationSuite._
sc = new SparkContext("local[2]", "test interruptible iterator")

// Increase the number of elements to be proceeded to avoid this test being flaky.
val numElements = 10000
val taskCompletedSem = new Semaphore(0)

sc.addSparkListener(new SparkListener {
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
// release taskCancelledSemaphore when cancelTasks event has been posted
if (stageCompleted.stageInfo.stageId == 1) {
taskCancelledSemaphore.release(1000)
taskCancelledSemaphore.release(numElements)
}
}

Expand All @@ -349,36 +351,39 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft
}
})

val f = sc.parallelize(1 to 1000).map { i => (i, i) }
// Explicitly disable interrupt task thread on cancelling tasks, so the task thread can only be
// interrupted by `InterruptibleIterator`.
sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false")

val f = sc.parallelize(1 to numElements).map { i => (i, i) }
.repartitionAndSortWithinPartitions(new HashPartitioner(1))
.mapPartitions { iter =>
taskStartedSemaphore.release()
iter
}.foreachAsync { x =>
if (x._1 >= 10) {
// This block of code is partially executed. It will be blocked when x._1 >= 10 and the
// next iteration will be cancelled if the source iterator is interruptible. Then in this
// case, the maximum num of increment would be 10(|1...10|)
taskCancelledSemaphore.acquire()
}
// Block this code from being executed, until the job get cancelled. In this case, if the
// source iterator is interruptible, the max number of increment should be under
// `numElements`.
taskCancelledSemaphore.acquire()
executionOfInterruptibleCounter.getAndIncrement()
}

taskStartedSemaphore.acquire()
// Job is cancelled when:
// 1. task in reduce stage has been started, guaranteed by previous line.
// 2. task in reduce stage is blocked after processing at most 10 records as
// taskCancelledSemaphore is not released until cancelTasks event is posted
// After job being cancelled, task in reduce stage will be cancelled and no more iteration are
// executed.
// 2. task in reduce stage is blocked as taskCancelledSemaphore is not released until
// JobCancelled event is posted.
// After job being cancelled, task in reduce stage will be cancelled asynchronously, thus
// partial of the inputs should not get processed (It's very unlikely that Spark can process
// 10000 elements between JobCancelled is posted and task is really killed).
f.cancel()

val e = intercept[SparkException](f.get()).getCause
assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))

// Make sure tasks are indeed completed.
taskCompletedSem.acquire()
assert(executionOfInterruptibleCounter.get() <= 10)
assert(executionOfInterruptibleCounter.get() < numElements)
}

def testCount() {
Expand Down
92 changes: 75 additions & 17 deletions core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala
Expand Up @@ -22,6 +22,7 @@ import java.lang.Thread.UncaughtExceptionHandler
import java.nio.ByteBuffer
import java.util.Properties
import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.concurrent.atomic.AtomicBoolean

import scala.collection.mutable.Map
import scala.concurrent.duration._
Expand Down Expand Up @@ -139,7 +140,7 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
// the fetch failure. The executor should still tell the driver that the task failed due to a
// fetch failure, not a generic exception from user code.
val inputRDD = new FetchFailureThrowingRDD(sc)
val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false)
val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false, interrupt = false)
val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array())
val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array()
val task = new ResultTask(
Expand Down Expand Up @@ -173,17 +174,48 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
}

test("SPARK-19276: OOMs correctly handled with a FetchFailure") {
val (failReason, uncaughtExceptionHandler) = testFetchFailureHandling(true)
assert(failReason.isInstanceOf[ExceptionFailure])
val exceptionCaptor = ArgumentCaptor.forClass(classOf[Throwable])
verify(uncaughtExceptionHandler).uncaughtException(any(), exceptionCaptor.capture())
assert(exceptionCaptor.getAllValues.size === 1)
assert(exceptionCaptor.getAllValues().get(0).isInstanceOf[OutOfMemoryError])
}

test("SPARK-23816: interrupts are not masked by a FetchFailure") {
// If killing the task causes a fetch failure, we still treat it as a task that was killed,
// as the fetch failure could easily be caused by interrupting the thread.
val (failReason, _) = testFetchFailureHandling(false)
assert(failReason.isInstanceOf[TaskKilled])
}

/**
* Helper for testing some cases where a FetchFailure should *not* get sent back, because its
* superceded by another error, either an OOM or intentionally killing a task.
* @param oom if true, throw an OOM after the FetchFailure; else, interrupt the task after the
* FetchFailure
*/
private def testFetchFailureHandling(
oom: Boolean): (TaskFailedReason, UncaughtExceptionHandler) = {
// when there is a fatal error like an OOM, we don't do normal fetch failure handling, since it
// may be a false positive. And we should call the uncaught exception handler.
// SPARK-23816 also handle interrupts the same way, as killing an obsolete speculative task
// does not represent a real fetch failure.
val conf = new SparkConf().setMaster("local").setAppName("executor suite test")
sc = new SparkContext(conf)
val serializer = SparkEnv.get.closureSerializer.newInstance()
val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size

// Submit a job where a fetch failure is thrown, but then there is an OOM. We should treat
// the fetch failure as a false positive, and just do normal OOM handling.
// Submit a job where a fetch failure is thrown, but then there is an OOM or interrupt. We
// should treat the fetch failure as a false positive, and do normal OOM or interrupt handling.
val inputRDD = new FetchFailureThrowingRDD(sc)
val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = true)
if (!oom) {
// we are trying to setup a case where a task is killed after a fetch failure -- this
// is just a helper to coordinate between the task thread and this thread that will
// kill the task
ExecutorSuiteHelper.latches = new ExecutorSuiteHelper()
}
val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = oom, interrupt = !oom)
val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array())
val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array()
val task = new ResultTask(
Expand All @@ -200,15 +232,8 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
val serTask = serializer.serialize(task)
val taskDescription = createFakeTaskDescription(serTask)

val (failReason, uncaughtExceptionHandler) =
runTaskGetFailReasonAndExceptionHandler(taskDescription)
// make sure the task failure just looks like a OOM, not a fetch failure
assert(failReason.isInstanceOf[ExceptionFailure])
val exceptionCaptor = ArgumentCaptor.forClass(classOf[Throwable])
verify(uncaughtExceptionHandler).uncaughtException(any(), exceptionCaptor.capture())
assert(exceptionCaptor.getAllValues.size === 1)
assert(exceptionCaptor.getAllValues.get(0).isInstanceOf[OutOfMemoryError])
}
runTaskGetFailReasonAndExceptionHandler(taskDescription, killTask = !oom)
}

test("Gracefully handle error in task deserialization") {
val conf = new SparkConf
Expand Down Expand Up @@ -257,22 +282,39 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
}

private def runTaskAndGetFailReason(taskDescription: TaskDescription): TaskFailedReason = {
runTaskGetFailReasonAndExceptionHandler(taskDescription)._1
runTaskGetFailReasonAndExceptionHandler(taskDescription, false)._1
}

private def runTaskGetFailReasonAndExceptionHandler(
taskDescription: TaskDescription): (TaskFailedReason, UncaughtExceptionHandler) = {
taskDescription: TaskDescription,
killTask: Boolean): (TaskFailedReason, UncaughtExceptionHandler) = {
val mockBackend = mock[ExecutorBackend]
val mockUncaughtExceptionHandler = mock[UncaughtExceptionHandler]
var executor: Executor = null
val timedOut = new AtomicBoolean(false)
try {
executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true,
uncaughtExceptionHandler = mockUncaughtExceptionHandler)
// the task will be launched in a dedicated worker thread
executor.launchTask(mockBackend, taskDescription)
if (killTask) {
val killingThread = new Thread("kill-task") {
override def run(): Unit = {
// wait to kill the task until it has thrown a fetch failure
if (ExecutorSuiteHelper.latches.latch1.await(10, TimeUnit.SECONDS)) {
// now we can kill the task
executor.killAllTasks(true, "Killed task, eg. because of speculative execution")
} else {
timedOut.set(true)
}
}
}
killingThread.start()
}
eventually(timeout(5.seconds), interval(10.milliseconds)) {
assert(executor.numRunningTasks === 0)
}
assert(!timedOut.get(), "timed out waiting to be ready to kill tasks")
} finally {
if (executor != null) {
executor.stop()
Expand All @@ -282,8 +324,9 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug
val statusCaptor = ArgumentCaptor.forClass(classOf[ByteBuffer])
orderedMock.verify(mockBackend)
.statusUpdate(meq(0L), meq(TaskState.RUNNING), statusCaptor.capture())
val finalState = if (killTask) TaskState.KILLED else TaskState.FAILED
orderedMock.verify(mockBackend)
.statusUpdate(meq(0L), meq(TaskState.FAILED), statusCaptor.capture())
.statusUpdate(meq(0L), meq(finalState), statusCaptor.capture())
// first statusUpdate for RUNNING has empty data
assert(statusCaptor.getAllValues().get(0).remaining() === 0)
// second update is more interesting
Expand Down Expand Up @@ -321,7 +364,8 @@ class SimplePartition extends Partition {
class FetchFailureHidingRDD(
sc: SparkContext,
val input: FetchFailureThrowingRDD,
throwOOM: Boolean) extends RDD[Int](input) {
throwOOM: Boolean,
interrupt: Boolean) extends RDD[Int](input) {
override def compute(split: Partition, context: TaskContext): Iterator[Int] = {
val inItr = input.compute(split, context)
try {
Expand All @@ -330,6 +374,15 @@ class FetchFailureHidingRDD(
case t: Throwable =>
if (throwOOM) {
throw new OutOfMemoryError("OOM while handling another exception")
} else if (interrupt) {
// make sure our test is setup correctly
assert(TaskContext.get().asInstanceOf[TaskContextImpl].fetchFailed.isDefined)
// signal our test is ready for the task to get killed
ExecutorSuiteHelper.latches.latch1.countDown()
// then wait for another thread in the test to kill the task -- this latch
// is never actually decremented, we just wait to get killed.
ExecutorSuiteHelper.latches.latch2.await(10, TimeUnit.SECONDS)
throw new IllegalStateException("timed out waiting to be interrupted")
} else {
throw new RuntimeException("User Exception that hides the original exception", t)
}
Expand All @@ -352,6 +405,11 @@ private class ExecutorSuiteHelper {
@volatile var testFailedReason: TaskFailedReason = _
}

// helper for coordinating killing tasks
private object ExecutorSuiteHelper {
var latches: ExecutorSuiteHelper = null
}

private class NonDeserializableTask extends FakeTask(0, 0) with Externalizable {
def writeExternal(out: ObjectOutput): Unit = {}
def readExternal(in: ObjectInput): Unit = {
Expand Down
Expand Up @@ -165,7 +165,7 @@ object DecisionTreeClassifier extends DefaultParamsReadable[DecisionTreeClassifi
@Since("1.4.0")
class DecisionTreeClassificationModel private[ml] (
@Since("1.4.0")override val uid: String,
@Since("1.4.0")override val rootNode: Node,
@Since("1.4.0")override val rootNode: ClassificationNode,
@Since("1.6.0")override val numFeatures: Int,
@Since("1.5.0")override val numClasses: Int)
extends ProbabilisticClassificationModel[Vector, DecisionTreeClassificationModel]
Expand All @@ -178,7 +178,7 @@ class DecisionTreeClassificationModel private[ml] (
* Construct a decision tree classification model.
* @param rootNode Root node of tree, with other nodes attached.
*/
private[ml] def this(rootNode: Node, numFeatures: Int, numClasses: Int) =
private[ml] def this(rootNode: ClassificationNode, numFeatures: Int, numClasses: Int) =
this(Identifiable.randomUID("dtc"), rootNode, numFeatures, numClasses)

override def predict(features: Vector): Double = {
Expand Down Expand Up @@ -276,8 +276,9 @@ object DecisionTreeClassificationModel extends MLReadable[DecisionTreeClassifica
val metadata = DefaultParamsReader.loadMetadata(path, sc, className)
val numFeatures = (metadata.metadata \ "numFeatures").extract[Int]
val numClasses = (metadata.metadata \ "numClasses").extract[Int]
val root = loadTreeNodes(path, metadata, sparkSession)
val model = new DecisionTreeClassificationModel(metadata.uid, root, numFeatures, numClasses)
val root = loadTreeNodes(path, metadata, sparkSession, isClassification = true)
val model = new DecisionTreeClassificationModel(metadata.uid,
root.asInstanceOf[ClassificationNode], numFeatures, numClasses)
DefaultParamsReader.getAndSetParams(model, metadata)
model
}
Expand All @@ -292,9 +293,10 @@ object DecisionTreeClassificationModel extends MLReadable[DecisionTreeClassifica
require(oldModel.algo == OldAlgo.Classification,
s"Cannot convert non-classification DecisionTreeModel (old API) to" +
s" DecisionTreeClassificationModel (new API). Algo is: ${oldModel.algo}")
val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures)
val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures, isClassification = true)
val uid = if (parent != null) parent.uid else Identifiable.randomUID("dtc")
// Can't infer number of features from old model, so default to -1
new DecisionTreeClassificationModel(uid, rootNode, numFeatures, -1)
new DecisionTreeClassificationModel(uid,
rootNode.asInstanceOf[ClassificationNode], numFeatures, -1)
}
}

0 comments on commit b9d99f7

Please sign in to comment.