diff options
Diffstat (limited to 'core/src/main/scala/org/apache')
24 files changed, 108 insertions, 110 deletions
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 10fae5af9f..ccffcc356c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -29,8 +29,7 @@ import akka.pattern.ask import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{MetadataCleanerType, Utils, MetadataCleaner, TimeStampedHashMap} - +import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int, requester: String) @@ -53,7 +52,7 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster private[spark] class MapOutputTracker extends Logging { - private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + private val timeout = AkkaUtils.askTimeout // Set to the MapOutputTrackerActor living on the driver var trackerActor: Either[ActorRef, ActorSelection] = _ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index c47657f512..037cd1c774 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -125,6 +125,8 @@ JavaRDDLike[T, JavaRDD[T]] { */ def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) + + override def toString = rdd.toString } object JavaRDD { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 9e912d3adb..f344804b4c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -245,6 +245,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** + * Return an array that contains all of the elements in a specific partition of this RDD. + */ + def collectPartitions(partitionIds: Array[Int]): Array[JList[T]] = { + // This is useful for implementing `take` from other language frontends + // like Python where the data is serialized. + import scala.collection.JavaConversions._ + val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds, true) + res.map(x => new java.util.ArrayList(x.toSeq)).toArray + } + + /** * Reduces the elements of this RDD using the specified commutative and associative binary operator. */ def reduce(f: JFunction2[T, T, T]): T = rdd.reduce(f) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index a659cc06c2..ca42c76928 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -235,10 +235,6 @@ private[spark] object PythonRDD { file.close() } - def takePartition[T](rdd: RDD[T], partition: Int): Iterator[T] = { - implicit val cm : ClassTag[T] = rdd.elementClassTag - rdd.context.runJob(rdd, ((x: Iterator[T]) => x.toArray), Seq(partition), true).head.iterator - } } private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala index 4d95efa73a..953755e40d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala @@ -23,14 +23,14 @@ import scala.concurrent.duration._ import scala.concurrent.Await import akka.actor._ -import akka.pattern.AskTimeoutException import akka.pattern.ask -import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} +import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} import org.apache.spark.{SparkException, Logging} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master +import org.apache.spark.util.AkkaUtils /** @@ -178,7 +178,7 @@ private[spark] class Client( def stop() { if (actor != null) { try { - val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val timeout = AkkaUtils.askTimeout val future = actor.ask(StopClient)(timeout) Await.result(future, timeout) } catch { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index c627dd3806..eebd0794b8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -18,19 +18,16 @@ package org.apache.spark.deploy.master import java.text.SimpleDateFormat -import java.util.concurrent.TimeUnit import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ -import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor._ import akka.pattern.ask -import akka.remote._ +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import akka.util.Timeout import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -38,7 +35,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.util.{AkkaUtils, Utils} private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher @@ -64,8 +61,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val waitingApps = new ArrayBuffer[ApplicationInfo] val completedApps = new ArrayBuffer[ApplicationInfo] - var firstApp: Option[ApplicationInfo] = None - Utils.checkHost(host, "Expected hostname") val masterMetricsSystem = MetricsSystem.createMetricsSystem("master") @@ -444,14 +439,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act idToApp(app.id) = app actorToApp(app.driver) = app addressToApp(appAddress) = app - if (firstApp == None) { - firstApp = Some(app) - } - // TODO: What is firstApp?? Can we remove it? - val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray - if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= app.desc.memoryPerSlave)) { - logWarning("Could not find any workers with enough memory for " + firstApp.get.id) - } waitingApps += app } @@ -537,12 +524,10 @@ private[spark] object Master { def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) - val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName) - val timeoutDuration: FiniteDuration = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, TimeUnit.SECONDS) - implicit val timeout = Timeout(timeoutDuration) - val respFuture = actor ? RequestWebUIPort // ask pattern - val resp = Await.result(respFuture, timeoutDuration).asInstanceOf[WebUIPortResponse] + val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) + val timeout = AkkaUtils.askTimeout + val respFuture = actor.ask(RequestWebUIPort)(timeout) + val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] (actorSystem, boundPort, resp.webUIBoundPort) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala index 81e15c534f..6cc7fd2ff4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala @@ -18,12 +18,12 @@ package org.apache.spark.deploy.master import scala.collection.JavaConversions._ -import scala.concurrent.ops._ -import org.apache.spark.Logging import org.apache.zookeeper._ -import org.apache.zookeeper.data.Stat import org.apache.zookeeper.Watcher.Event.KeeperState +import org.apache.zookeeper.data.Stat + +import org.apache.spark.Logging /** * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry @@ -33,7 +33,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState * informed via zkDown(). * * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many - * times or a semantic exception is thrown (e.g.., "node already exists"). + * times or a semantic exception is thrown (e.g., "node already exists"). */ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging { val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "") @@ -103,6 +103,7 @@ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) ext connectToZooKeeper() case KeeperState.Disconnected => logWarning("ZooKeeper disconnected, will retry...") + case s => // Do nothing } } } @@ -179,7 +180,7 @@ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) ext } catch { case e: KeeperException.NoNodeException => throw e case e: KeeperException.NodeExistsException => throw e - case e if n > 0 => + case e: Exception if n > 0 => logError("ZooKeeper exception, " + n + " more retries...", e) Thread.sleep(RETRY_WAIT_MILLIS) retry(fn, n-1) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 7809013e83..7d535b08de 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -21,8 +21,8 @@ import akka.actor.ActorRef import org.apache.zookeeper._ import org.apache.zookeeper.Watcher.Event.EventType -import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.Logging +import org.apache.spark.deploy.master.MasterMessages._ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String) extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { @@ -105,7 +105,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, mas // We found a different master file pointing to this process. // This can happen in the following two cases: // (1) The master process was restarted on the same node. - // (2) The ZK server died between creating the node and returning the name of the node. + // (2) The ZK server died between creating the file and returning the name of the file. // For this case, we will end up creating a second file, and MUST explicitly delete the // first one, since our ZK session is still open. // Note that this deletion will cause a NodeDeleted event to be fired so we check again for diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 3b983c19eb..dbb0cb90f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -17,32 +17,28 @@ package org.apache.spark.deploy.master.ui +import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask - -import scala.concurrent.Await -import scala.concurrent.duration._ - import javax.servlet.http.HttpServletRequest - import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.JsonProtocol +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils private[spark] class ApplicationPage(parent: MasterWebUI) { val master = parent.masterActorRef - implicit val timeout = parent.timeout + val timeout = parent.timeout /** Executor details for a particular application */ def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) @@ -53,7 +49,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { def render(request: HttpServletRequest): Seq[Node] = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) val app = state.activeApps.find(_.id == appId).getOrElse({ state.completedApps.find(_.id == appId).getOrElse(null) }) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 65e7a14e7a..4ef762892c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -17,37 +17,33 @@ package org.apache.spark.deploy.master.ui -import javax.servlet.http.HttpServletRequest - +import scala.concurrent.Await import scala.xml.Node -import scala.concurrent.Await import akka.pattern.ask -import scala.concurrent.duration._ - +import javax.servlet.http.HttpServletRequest import net.liftweb.json.JsonAST.JValue -import org.apache.spark.deploy.DeployWebUI +import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} -import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils private[spark] class IndexPage(parent: MasterWebUI) { val master = parent.masterActorRef - implicit val timeout = parent.timeout + val timeout = parent.timeout def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) JsonProtocol.writeMasterState(state) } /** Index view listing applications and executors */ def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] - val state = Await.result(stateFuture, 30 seconds) + val state = Await.result(stateFuture, timeout) val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory") val workers = state.workers.sortBy(_.id) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index a211ce2b42..9ab594b682 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,25 +17,21 @@ package org.apache.spark.deploy.master.ui -import scala.concurrent.duration._ - import javax.servlet.http.HttpServletRequest - import org.eclipse.jetty.server.{Handler, Server} -import org.apache.spark.{Logging} +import org.apache.spark.Logging import org.apache.spark.deploy.master.Master import org.apache.spark.ui.JettyUtils import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{AkkaUtils, Utils} /** * Web UI server for the standalone master. */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val timeout = AkkaUtils.askTimeout val host = Utils.localHostName() val port = requestedPort diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 1a768d501f..0d59048313 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -42,13 +42,13 @@ private[spark] class IndexPage(parent: WorkerWebUI) { def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] - val workerState = Await.result(stateFuture, 30 seconds) + val workerState = Await.result(stateFuture, timeout) JsonProtocol.writeWorkerState(workerState) } def render(request: HttpServletRequest): Seq[Node] = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] - val workerState = Await.result(stateFuture, 30 seconds) + val workerState = Await.result(stateFuture, timeout) val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs") val runningExecutorTable = diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 6c18a3c245..40d6bdb3fd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -19,17 +19,14 @@ package org.apache.spark.deploy.worker.ui import java.io.File -import scala.concurrent.duration._ - -import akka.util.Timeout import javax.servlet.http.HttpServletRequest +import org.eclipse.jetty.server.{Handler, Server} import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.Utils -import org.eclipse.jetty.server.{Handler, Server} +import org.apache.spark.util.{AkkaUtils, Utils} /** * Web UI server for the standalone worker. @@ -37,8 +34,7 @@ import org.eclipse.jetty.server.{Handler, Server} private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) extends Logging { - implicit val timeout = Timeout( - Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")) + val timeout = AkkaUtils.askTimeout val host = Utils.localHostName() val port = requestedPort.getOrElse( System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 0b0a60ee60..0f19d7a96b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -222,18 +222,22 @@ private[spark] class Executor( return } + val resultSer = SparkEnv.get.serializer.newInstance() + val beforeSerialization = System.currentTimeMillis() + val valueBytes = resultSer.serialize(value) + val afterSerialization = System.currentTimeMillis() + for (m <- task.metrics) { m.hostname = Utils.localHostName() m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt m.jvmGCTime = gcTime - startGCTime + m.resultSerializationTime = (afterSerialization - beforeSerialization).toInt } - // TODO I'd also like to track the time it takes to serialize the task results, but that is - // huge headache, b/c we need to serialize the task metrics first. If TaskMetrics had a - // custom serialized format, we could just change the relevants bytes in the byte buffer + val accumUpdates = Accumulators.values - val directResult = new DirectTaskResult(value, accumUpdates, task.metrics.getOrElse(null)) + val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.getOrElse(null)) val serializedDirectResult = ser.serialize(directResult) logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit) val serializedResult = { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index c0ce46e379..bb1471d9ee 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -44,6 +44,11 @@ class TaskMetrics extends Serializable { var jvmGCTime: Long = _ /** + * Amount of time spent serializing the task result + */ + var resultSerializationTime: Long = _ + + /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here */ var shuffleReadMetrics: Option[ShuffleReadMetrics] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 7e468d0d67..e80cc6b0f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -35,18 +35,15 @@ case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Se /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] -class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) +class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics) extends TaskResult[T] with Externalizable { - def this() = this(null.asInstanceOf[T], null, null) + def this() = this(null.asInstanceOf[ByteBuffer], null, null) override def writeExternal(out: ObjectOutput) { - val objectSer = SparkEnv.get.serializer.newInstance() - val bb = objectSer.serialize(value) - - out.writeInt(bb.remaining()) - Utils.writeByteBuffer(bb, out) + out.writeInt(valueBytes.remaining); + Utils.writeByteBuffer(valueBytes, out) out.writeInt(accumUpdates.size) for ((key, value) <- accumUpdates) { @@ -58,12 +55,10 @@ class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var me override def readExternal(in: ObjectInput) { - val objectSer = SparkEnv.get.serializer.newInstance() - val blen = in.readInt() val byteVal = new Array[Byte](blen) in.readFully(byteVal) - value = objectSer.deserialize(ByteBuffer.wrap(byteVal)) + valueBytes = ByteBuffer.wrap(byteVal) val numUpdates = in.readInt if (numUpdates == 0) { @@ -76,4 +71,9 @@ class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var me } metrics = in.readObject().asInstanceOf[TaskMetrics] } + + def value(): T = { + val resultSer = SparkEnv.get.serializer.newInstance() + return resultSer.deserialize(valueBytes) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala index 94961790df..bf494aa64d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala @@ -529,10 +529,10 @@ private[spark] class ClusterTaskSetManager( addPendingTask(index) if (state != TaskState.KILLED) { numFailures(index) += 1 - if (numFailures(index) > MAX_TASK_FAILURES) { - logError("Task %s:%d failed more than %d times; aborting job".format( + if (numFailures(index) >= MAX_TASK_FAILURES) { + logError("Task %s:%d failed %d times; aborting job".format( taskSet.id, index, MAX_TASK_FAILURES)) - abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) + abort("Task %s:%d failed %d times".format(taskSet.id, index, MAX_TASK_FAILURES)) } } } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index f5e8766f6d..7e22c843bf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -27,10 +27,10 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{SparkException, Logging, TaskState} +import org.apache.spark.{Logging, SparkException, TaskState} import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{AkkaUtils, Utils} /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -47,6 +47,8 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) + private val timeout = AkkaUtils.askTimeout + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] private val executorAddress = new HashMap[String, Address] @@ -172,10 +174,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } - private val timeout = { - Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - } - def stopExecutors() { try { if (driverActor != null) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e05b842476..e1d68ef592 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -18,7 +18,6 @@ package org.apache.spark.storage import scala.concurrent.{Await, Future} -import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ @@ -26,15 +25,17 @@ import akka.pattern.ask import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.AkkaUtils -private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { +private[spark] +class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" - val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + val timeout = AkkaUtils.askTimeout /** Remove a dead executor from the driver actor. This is only called on the driver side. */ def removeExecutor(execId: String) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 154a3980e9..21022e1cfb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -21,17 +21,15 @@ import java.util.{HashMap => JHashMap} import scala.collection.mutable import scala.collection.JavaConversions._ +import scala.concurrent.Future +import scala.concurrent.duration._ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask -import scala.concurrent.duration._ -import scala.concurrent.Future - import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.Utils - +import org.apache.spark.util.{AkkaUtils, Utils} /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -50,8 +48,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging { // Mapping from block id to the set of block managers that have the block. private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]] - val akkaTimeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") + private val akkaTimeout = AkkaUtils.askTimeout initLogging() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index d8a6c9e2dc..8dcfeacb60 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -86,7 +86,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val taskHeaders: Seq[String] = Seq("Task Index", "Task ID", "Status", "Locality Level", "Executor", "Launch Time") ++ - Seq("Duration", "GC Time") ++ + Seq("Duration", "GC Time", "Result Ser Time") ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ Seq("Errors") @@ -101,6 +101,11 @@ private[spark] class StagePage(parent: JobProgressUI) { None } else { + val serializationTimes = validTasks.map{case (info, metrics, exception) => + metrics.get.resultSerializationTime.toDouble} + val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes).get.getQuantiles().map( + ms => parent.formatDuration(ms.toLong)) + val serviceTimes = validTasks.map{case (info, metrics, exception) => metrics.get.executorRunTime.toDouble} val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( @@ -149,6 +154,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) val listings: Seq[Seq[String]] = Seq( + serializationQuantiles, serviceQuantiles, gettingResultQuantiles, schedulerDelayQuantiles, @@ -184,6 +190,7 @@ private[spark] class StagePage(parent: JobProgressUI) { val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) + val serializationTime = metrics.map(m => m.resultSerializationTime).getOrElse(0L) val maybeShuffleRead = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead} val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") @@ -211,6 +218,9 @@ private[spark] class StagePage(parent: JobProgressUI) { <td sorttable_customkey={gcTime.toString}> {if (gcTime > 0) parent.formatDuration(gcTime) else ""} </td> + <td sorttable_customkey={serializationTime.toString}> + {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + </td> {if (shuffleRead) { <td sorttable_customkey={shuffleReadSortable}> {shuffleReadReadable} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index a5446b3fc3..39f422dd6b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -28,9 +28,6 @@ import org.apache.spark.ui.JettyUtils._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { - implicit val timeout = Duration.create( - System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") - val indexPage = new IndexPage(this) val rddPage = new RDDPage(this) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 74133cef6c..1c8b51b8bc 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.util +import scala.concurrent.duration.{Duration, FiniteDuration} + import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory @@ -84,4 +86,8 @@ private[spark] object AkkaUtils { (actorSystem, boundPort) } + /** Returns the default Spark timeout to use for Akka ask operations. */ + def askTimeout: FiniteDuration = { + Duration.create(System.getProperty("spark.akka.askTimeout", "30").toLong, "seconds") + } } diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index 0b51c23f7b..a38329df03 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -34,6 +34,8 @@ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) override def iterator: Iterator[A] = underlying.iterator.asScala + override def size: Int = underlying.size + override def ++=(xs: TraversableOnce[A]): this.type = { xs.foreach { this += _ } this |