Github user JoshRosen commented on a diff in the pull request:
https://github.com/apache/spark/pull/4155#discussion_r23564616
--- Diff:
core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala ---
@@ -0,0 +1,252 @@
+/*
+ * 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 java.util.concurrent.{ExecutorService, TimeUnit, Executors,
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}
+
+private[spark] sealed trait OutputCommitCoordinationMessage
+
+private[spark] case class StageStarted(stage: Int, partitionIds: Seq[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,
+ partId: Int,
+ taskAttempt: Long)
+ extends OutputCommitCoordinationMessage with Serializable
+
+private[spark] case class TaskCompleted(
+ stage: Int,
+ task: Long,
+ partId: Int,
+ 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 {
+
+ private type StageId = Int
+ private type PartitionId = Int
+ private type TaskId = Long
+ private type TaskAttemptId = Long
+
+ // Wrapper for an int option that allows it to be locked via a
synchronized block
+ // while still setting option itself to Some(...) or None.
+ private class LockableAttemptId(var value: Option[TaskAttemptId])
+
+ private type CommittersByStageHashMap =
+ ConcurrentHashMap[StageId, ScalaImmutableMap[PartitionId,
LockableAttemptId]]
+
+ // Initialized by SparkEnv
+ private var coordinatorActor: Option[ActorRef] = None
+ private val timeout = AkkaUtils.askTimeout(conf)
+ private val maxAttempts = AkkaUtils.numRetries(conf)
+ private val retryInterval = AkkaUtils.retryWaitMs(conf)
+ private val authorizedCommittersByStage = new
CommittersByStageHashMap().asScala
+
+ private var executorRequestHandlingThreadPool: Option[ExecutorService] =
None
+
+ def stageStart(stage: StageId, partitionIds: Seq[Int]): Unit = {
+ sendToActor(StageStarted(stage, partitionIds))
+ }
+
+ def stageEnd(stage: StageId): Unit = {
+ sendToActor(StageEnded(stage))
+ }
+
+ def canCommit(
+ stage: StageId,
+ task: TaskId,
+ partId: PartitionId,
+ attempt: TaskAttemptId): Boolean = {
+ askActor(AskPermissionToCommitOutput(stage, task, partId, attempt))
+ }
+
+ def taskCompleted(
+ stage: StageId,
+ task: TaskId,
+ partId: PartitionId,
+ attempt: TaskAttemptId,
+ successful: Boolean): Unit = {
+ sendToActor(TaskCompleted(stage, task, partId, attempt, successful))
+ }
+
+ def stop(): Unit = {
+ executorRequestHandlingThreadPool.foreach { pool =>
+ pool.shutdownNow()
+ pool.awaitTermination(10, TimeUnit.SECONDS)
+ }
+ sendToActor(StopCoordinator)
+ coordinatorActor = None
+ executorRequestHandlingThreadPool = None
+ authorizedCommittersByStage.clear
+ }
+
+ def initialize(actor: ActorRef, isDriver: Boolean): Unit = {
+ coordinatorActor = Some(actor)
+ executorRequestHandlingThreadPool = {
+ if (isDriver) {
+ Some(Executors.newFixedThreadPool(4))
--- End diff --
Do you think that the thread-pool here is going to buy us a performance
benefit, or will we experience a bottleneck in the single-threaded message
dispatch that feeds into the thread pool?
Also, if we do decide to use thread pools, we should use the Utils function
that makes them create daemon threads:
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/util/Utils.scala#L784
---
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]