Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4155#discussion_r23434036
  
    --- Diff: 
core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala ---
    @@ -0,0 +1,172 @@
    +/*
    + * 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.scheduler
    +
    +import scala.collection.mutable
    +import scala.concurrent.duration.FiniteDuration
    +
    +import akka.actor.{PoisonPill, ActorRef, Actor}
    +
    +import org.apache.spark.{SparkConf, Logging}
    +import org.apache.spark.util.{AkkaUtils, ActorLogReceive}
    +
    +private[spark] sealed trait OutputCommitCoordinationMessage extends 
Serializable
    +
    +private[spark] case class StageStarted(stage: Int) extends 
OutputCommitCoordinationMessage
    +private[spark] case class StageEnded(stage: Int) extends 
OutputCommitCoordinationMessage
    +private[spark] case object StopCoordinator extends 
OutputCommitCoordinationMessage
    +
    +private[spark] case class AskPermissionToCommitOutput(
    +    stage: Int,
    +    task: Long,
    +    taskAttempt: Long)
    +    extends OutputCommitCoordinationMessage
    +
    +private[spark] case class TaskCompleted(
    +    stage: Int,
    +    task: Long,
    +    attempt: Long,
    +    successful: Boolean)
    +    extends OutputCommitCoordinationMessage
    +
    +/**
    + * Authority that decides whether tasks can commit output to HDFS.
    + *
    + * This lives on the driver, but the actor allows the tasks that commit
    + * to Hadoop to invoke it.
    + */
    +private[spark] class OutputCommitCoordinator(conf: SparkConf) extends 
Logging {
    +
    +  // Initialized by SparkEnv
    +  var coordinatorActor: ActorRef = _
    +  val timeout = AkkaUtils.askTimeout(conf)
    +  val maxAttempts = AkkaUtils.numRetries(conf)
    +  val retryInterval = AkkaUtils.retryWaitMs(conf)
    +
    +  private type StageId = Int
    +  private type TaskId = Long
    +  private type TaskAttemptId = Long
    +
    +  private val authorizedCommittersByStage:
    +  mutable.Map[StageId, mutable.Map[TaskId, TaskAttemptId]] = 
mutable.HashMap()
    +
    +  def stageStart(stage: StageId) {
    +    coordinatorActor ! StageStarted(stage)
    +  }
    +  def stageEnd(stage: StageId) {
    +    coordinatorActor ! StageEnded(stage)
    +  }
    +
    +  def canCommit(
    +      stage: StageId,
    +      task: TaskId,
    +      attempt: TaskAttemptId): Boolean = {
    +    AkkaUtils.askWithReply(AskPermissionToCommitOutput(stage, task, 
attempt),
    +      coordinatorActor, maxAttempts, retryInterval, timeout)
    +  }
    +
    +  def taskCompleted(
    +      stage: StageId,
    +      task: TaskId,
    +      attempt: TaskAttemptId,
    +      successful: Boolean) {
    +    coordinatorActor ! TaskCompleted(stage, task, attempt, successful)
    +  }
    +
    +  def stop() {
    +    val stopped = AkkaUtils.askWithReply[Boolean](StopCoordinator, 
coordinatorActor, timeout)
    +    if (!stopped) {
    +      logWarning("Expected true from stopping output coordinator actor, 
but got false!")
    +    }
    +    authorizedCommittersByStage.foreach(_._2.clear)
    +    authorizedCommittersByStage.clear
    +  }
    +
    +  private def handleStageStart(stage: StageId): Unit = {
    +    authorizedCommittersByStage(stage) = mutable.HashMap[TaskId, 
TaskAttemptId]()
    +  }
    +
    +  private def handleStageEnd(stage: StageId): Unit = {
    +    authorizedCommittersByStage.remove(stage)
    +  }
    +
    +  private def handleAskPermissionToCommit(
    +      stage: StageId,
    +      task: TaskId,
    +      attempt: TaskAttemptId):
    +      Boolean = {
    +    if (!authorizedCommittersByStage.contains(stage)) {
    +      logDebug(s"Stage $stage has completed, so not allowing task attempt 
$attempt to commit")
    +      return false
    +    }
    +    val authorizedCommitters = authorizedCommittersByStage(stage)
    +    if (authorizedCommitters.contains(task)) {
    +      val existingCommitter = authorizedCommitters(task)
    +      logDebug(s"Denying $attempt to commit for stage=$stage, task=$task; 
" +
    +        s"existingCommitter = $existingCommitter")
    +      false
    +    } else {
    +      logDebug(s"Authorizing $attempt to commit for stage=$stage, 
task=$task")
    +      authorizedCommitters(task) = attempt
    +      true
    +    }
    +  }
    +
    +  private def handleTaskCompletion(
    +      stage: StageId,
    +      task: TaskId,
    +      attempt: TaskAttemptId,
    +      successful: Boolean): Unit = {
    +    if (!authorizedCommittersByStage.contains(stage)) {
    +      logDebug(s"Ignoring task completion for completed stage")
    +      return
    +    }
    +    val authorizedCommitters = authorizedCommittersByStage(stage)
    +    if (authorizedCommitters.get(task) == Some(attempt) && !successful) {
    +      logDebug(s"Authorized committer $attempt (stage=$stage, task=$task) 
failed; clearing lock")
    +      // The authorized committer failed; clear the lock so future 
attempts can commit their output
    +      authorizedCommitters.remove(task)
    +    }
    +  }
    +
    +}
    +
    +private[spark] object OutputCommitCoordinator {
    +
    +  class OutputCommitCoordinatorActor(outputCommitCoordinator: 
OutputCommitCoordinator)
    --- End diff --
    
    What's the motivation for defining this class inside of the 
`OutputCommitCoordinator` and having a `createActor` method instead of a 
regular constructor call?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to