Skip to content

Commit

Permalink
Throwing exception if SparkHadoopWriter commit denied
Browse files Browse the repository at this point in the history
However, the Executor will not treat this like any other
ExceptionFailure, but rather send a specific TaskEndReason to the
driver. The driver simply ignores the task that failed to commit after
logging. If the task that attempted to commit first fails, it will be
resubmitted.

TODO unit tests, the current unit tests don't capture this workflow all
the way down to the Executor level.
  • Loading branch information
mccheah committed Jan 27, 2015
1 parent d431144 commit 1df2a91
Show file tree
Hide file tree
Showing 8 changed files with 76 additions and 11 deletions.
31 changes: 31 additions & 0 deletions core/src/main/scala/org/apache/spark/CommitDeniedException.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark

import org.apache.spark.annotation.DeveloperApi

/**
* :: DeveloperApi ::
* Exception thrown when a task attempts to commit output to Hadoop, but
* is denied by the driver.
*/
@DeveloperApi
class CommitDeniedException(msg: String, jobID: Int, splitID: Int, attemptID: Int)
extends Exception(msg) {
def toTaskEndReason(): TaskEndReason = new TaskCommitDenied(jobID, splitID, attemptID)
}
8 changes: 6 additions & 2 deletions core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
Original file line number Diff line number Diff line change
Expand Up @@ -122,10 +122,14 @@ class SparkHadoopWriter(@transient jobConf: JobConf)
}
}
} else {
logInfo(s"$taID: Not committed because DAGScheduler did not authorize commit")
val msg: String = s"$taID: Not committed because the driver did not authorize commit"
logInfo(msg)
throw new CommitDeniedException(msg, jobID, splitID, attemptID)
}
} else {
logInfo(s"No need to commit output of task because needsTaskCommit=false: ${taID.value}")
val msg: String = s"No need to commit output of task because needsTaskCommit=false: ${taID.value}"
logInfo(msg)
throw new CommitDeniedException(msg, jobID, splitID, attemptID)
}
}

Expand Down
14 changes: 14 additions & 0 deletions core/src/main/scala/org/apache/spark/TaskEndReason.scala
Original file line number Diff line number Diff line change
Expand Up @@ -146,6 +146,20 @@ case object TaskKilled extends TaskFailedReason {
override def toErrorString: String = "TaskKilled (killed intentionally)"
}

/**
* :: DeveloperApi ::
* Task requested the driver to commit, but was denied.
*/
@DeveloperApi
case class TaskCommitDenied(
jobID: Int,
splitID: Int,
attemptID: Int)
extends TaskFailedReason {
override def toErrorString: String = s"TaskCommitDenied (Driver denied task commit)" +
s" for job: $jobID, split: $splitID, attempt: $attemptID"
}

/**
* :: DeveloperApi ::
* The task failed because the executor that it was running on was lost. This may happen because
Expand Down
5 changes: 5 additions & 0 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Original file line number Diff line number Diff line change
Expand Up @@ -248,6 +248,11 @@ private[spark] class Executor(
execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
}

case cDE: CommitDeniedException => {
val reason = cDE.toTaskEndReason
execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
}

case t: Throwable => {
// Attempt to exit cleanly by informing the driver of our failure.
// If anything goes wrong (or this was a fatal exception), we will delegate to
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1082,6 +1082,9 @@ class DAGScheduler(
handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch))
}

case TaskCommitDenied(jobID, splitID, attemptID) =>
// Do nothing here, left up to the TaskScheduler to decide how to handle denied commits

case ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics) =>
// Do nothing here, left up to the TaskScheduler to decide how to handle user failures

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,15 @@

package org.apache.spark.scheduler

import java.util.concurrent.{ExecutorService, TimeUnit, Executors, ConcurrentHashMap}
import java.util.concurrent.{ExecutorService, TimeUnit, ConcurrentHashMap}

import scala.collection.{Map => ScalaImmutableMap}
import scala.collection.concurrent.{Map => ScalaConcurrentMap}
import scala.collection.convert.decorateAsScala._

import akka.actor.{ActorRef, Actor}

import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.util.{AkkaUtils, ActorLogReceive}
import org.apache.spark.util.{Utils, AkkaUtils, ActorLogReceive}

private[spark] sealed trait OutputCommitCoordinationMessage

Expand Down Expand Up @@ -119,7 +118,7 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging {
coordinatorActor = Some(actor)
executorRequestHandlingThreadPool = {
if (isDriver) {
Some(Executors.newFixedThreadPool(4))
Some(Utils.newDaemonFixedThreadPool(8, "OutputCommitCoordinator"))
} else {
None
}
Expand Down Expand Up @@ -174,8 +173,15 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf) extends Logging {
task: TaskId,
partId: PartitionId,
attempt: TaskAttemptId): Unit = {
executorRequestHandlingThreadPool.foreach(_.submit(
new AskCommitRunnable(requester, this, stage, task, partId, attempt)))
executorRequestHandlingThreadPool match {
case Some(threadPool) =>
threadPool.submit(new AskCommitRunnable(requester, this, stage, task, partId, attempt))
case None =>
logWarning("Got a request to commit output, but the OutputCommitCoordinator was already" +
" shut down. Request is being denied.")
requester ! false
}

}

private def handleTaskCompletion(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -646,6 +646,9 @@ private[spark] class TaskSetManager(
s"${ef.className} (${ef.description}) [duplicate $dupCount]")
}

case e: TaskCommitDenied =>
logWarning(failureReason)

case e: TaskFailedReason => // TaskResultLost, TaskKilled, and others
logWarning(failureReason)

Expand All @@ -657,7 +660,7 @@ private[spark] class TaskSetManager(
put(info.executorId, clock.getTime())
sched.dagScheduler.taskEnded(tasks(index), reason, null, null, info, taskMetrics)
addPendingTask(index)
if (!isZombie && state != TaskState.KILLED) {
if (!isZombie && state != TaskState.KILLED && !reason.isInstanceOf[TaskCommitDenied]) {
assert (null != failureReason)
numFailures(index) += 1
if (numFailures(index) >= maxTaskFailures) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,11 @@ package org.apache.spark.scheduler
import scala.collection.mutable.{ArrayBuffer, HashSet, HashMap, Map}
import scala.language.reflectiveCalls

import org.mockito.Mockito.mock
import org.scalatest.{BeforeAndAfter, FunSuiteLike}
import org.scalatest.concurrent.Timeouts
import org.scalatest.time.SpanSugar._

import org.mockito.Mockito.mock

import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
Expand Down

0 comments on commit 1df2a91

Please sign in to comment.