From 276c37a51c9a6188dbbe02754935540ace338dd1 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 22 Sep 2013 08:20:12 +0530 Subject: Akka 2.2 migration --- .../src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/deploy/client/Client.scala | 10 +++---- .../org/apache/spark/deploy/master/Master.scala | 31 ++++++++++++++++------ .../org/apache/spark/deploy/worker/Worker.scala | 12 ++++----- .../scala/org/apache/spark/executor/Executor.scala | 2 +- .../spark/executor/StandaloneExecutorBackend.scala | 30 ++++++++++++++------- .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../cluster/StandaloneSchedulerBackend.scala | 12 ++++----- .../mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../scala/org/apache/spark/util/AkkaUtils.scala | 23 ++++++++-------- 10 files changed, 75 insertions(+), 51 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 1e63b54b7a..a267407c67 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -169,7 +169,7 @@ object SparkEnv extends Logging { val driverHost: String = System.getProperty("spark.driver.host", "localhost") val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") - val url = "akka://spark@%s:%s/user/%s".format(driverHost, driverPort, name) + val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) actorSystem.actorFor(url) } 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 14a90934f6..164386782c 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 @@ -26,9 +26,7 @@ import akka.actor._ import akka.actor.Terminated import akka.pattern.AskTimeoutException import akka.pattern.ask -import akka.remote.RemoteClientDisconnected -import akka.remote.RemoteClientLifeCycleEvent -import akka.remote.RemoteClientShutdown +import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} import org.apache.spark.Logging import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -61,7 +59,7 @@ private[spark] class Client( master = context.actorFor(Master.toAkkaUrl(masterUrl)) masterAddress = master.path.address master ! RegisterApplication(appDescription) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { case e: Exception => @@ -99,12 +97,12 @@ private[spark] class Client( markDisconnected() context.stop(self) - case RemoteClientDisconnected(transport, address) if address == masterAddress => + case DisassociatedEvent(_, address, _) if address == masterAddress => logError("Connection to master failed; stopping client") markDisconnected() context.stop(self) - case RemoteClientShutdown(transport, address) if address == masterAddress => + case AssociationErrorEvent(_, _, address, _) if address == masterAddress => logError("Connection to master failed; stopping client") markDisconnected() context.stop(self) 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 2efd16bca0..cb0fe6a850 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 @@ -25,9 +25,8 @@ import scala.concurrent.Await import scala.concurrent.duration._ import akka.actor._ -import akka.actor.Terminated import akka.pattern.ask -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown} +import akka.remote._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} @@ -36,6 +35,22 @@ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} import akka.util.Timeout +import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed +import org.apache.spark.deploy.DeployMessages.KillExecutor +import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import scala.Some +import org.apache.spark.deploy.DeployMessages.WebUIPortResponse +import org.apache.spark.deploy.DeployMessages.LaunchExecutor +import org.apache.spark.deploy.DeployMessages.RegisteredApplication +import org.apache.spark.deploy.DeployMessages.RegisterWorker +import org.apache.spark.deploy.DeployMessages.ExecutorUpdated +import org.apache.spark.deploy.DeployMessages.MasterStateResponse +import org.apache.spark.deploy.DeployMessages.ExecutorAdded +import org.apache.spark.deploy.DeployMessages.RegisterApplication +import org.apache.spark.deploy.DeployMessages.ApplicationRemoved +import org.apache.spark.deploy.DeployMessages.Heartbeat +import org.apache.spark.deploy.DeployMessages.RegisteredWorker +import akka.actor.Terminated private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { @@ -81,7 +96,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def preStart() { logInfo("Starting Spark master at spark://" + host + ":" + port) // Listen for remote client disconnection events, since they don't go through Akka's watch() - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.start() import context.dispatcher context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) @@ -165,13 +180,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act actorToApp.get(actor).foreach(finishApplication) } - case RemoteClientDisconnected(transport, address) => { + case DisassociatedEvent(_, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) addressToApp.get(address).foreach(finishApplication) } - case RemoteClientShutdown(transport, address) => { + case AssociationErrorEvent(_, _, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was addressToWorker.get(address).foreach(removeWorker) addressToApp.get(address).foreach(finishApplication) @@ -376,11 +391,11 @@ private[spark] object Master { actorSystem.awaitTermination() } - /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ + /** Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ def toAkkaUrl(sparkUrl: String): String = { sparkUrl match { case sparkUrlRegex(host, port) => - "akka://%s@%s:%s/user/%s".format(systemName, host, port, actorName) + "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName) case _ => throw new SparkException("Invalid master URL: " + sparkUrl) } @@ -388,7 +403,7 @@ 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(new Master(host, boundPort, webUiPort)), name = actorName) + val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), name = actorName) val timeoutDuration = Duration.create( System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") implicit val timeout = Timeout(timeoutDuration) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index a0a9d1040a..1f04c1eea5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -25,9 +25,9 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} -import org.apache.spark.{Logging} +import org.apache.spark.Logging import org.apache.spark.deploy.ExecutorState import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -113,7 +113,7 @@ private[spark] class Worker( logInfo("Connecting to master " + masterUrl) master = context.actorFor(Master.toAkkaUrl(masterUrl)) master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) context.watch(master) // Doesn't work with remote actors, but useful for testing } @@ -165,7 +165,7 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + case _: Terminated | DisassociatedEvent | AssociationErrorEvent => masterDisconnected() case RequestWorkerState => { @@ -207,8 +207,8 @@ private[spark] object Worker { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port) - val actor = actorSystem.actorOf(Props(new Worker(host, boundPort, webUiPort, cores, memory, - masterUrl, workDir)), name = "Worker") + actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, + masterUrl, workDir), name = "Worker") (actorSystem, boundPort) } 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 ceae3b8289..99a4a95e82 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -105,7 +105,7 @@ private[spark] class Executor( SparkEnv.set(env) env.metricsSystem.registerSource(executorSource) - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size") + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size") // Start worker thread pool val threadPool = new ThreadPoolExecutor( diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala index 7839023868..46f0ef2cc6 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -19,13 +19,25 @@ package org.apache.spark.executor import java.nio.ByteBuffer -import akka.actor.{ActorRef, Actor, Props, Terminated} -import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected} +import akka.actor._ +import akka.remote._ import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask +import akka.remote.DisassociatedEvent +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisteredExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.LaunchTask +import akka.remote.AssociationErrorEvent +import akka.remote.DisassociatedEvent +import akka.actor.Terminated +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutor +import org.apache.spark.scheduler.cluster.StandaloneClusterMessages.RegisterExecutorFailed private[spark] class StandaloneExecutorBackend( @@ -40,14 +52,14 @@ private[spark] class StandaloneExecutorBackend( Utils.checkHostPort(hostPort, "Expected hostport") var executor: Executor = null - var driver: ActorRef = null + var driver: ActorSelection = null override def preStart() { logInfo("Connecting to driver: " + driverUrl) - driver = context.actorFor(driverUrl) + driver = context.actorSelection(driverUrl) driver ! RegisterExecutor(executorId, hostPort, cores) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - context.watch(driver) // Doesn't work with remote actors, but useful for testing + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + // context.watch(driver) // Doesn't work with remote actors, but useful for testing } override def receive = { @@ -69,7 +81,7 @@ private[spark] class StandaloneExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) => + case _: Terminated | DisassociatedEvent | AssociationErrorEvent => logError("Driver terminated or disconnected! Shutting down.") System.exit(1) } @@ -90,8 +102,8 @@ private[spark] object StandaloneExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) - val actor = actorSystem.actorOf( - Props(new StandaloneExecutorBackend(driverUrl, executorId, sparkHostPort, cores)), + actorSystem.actorOf( + Props(classOf[StandaloneExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") actorSystem.awaitTermination() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 9c49768c0c..fa83ae19d6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -42,7 +42,7 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index addfa077c1..49f668eb32 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -25,7 +25,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent} +import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._ @@ -53,7 +53,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) // Periodically revive offers to allow delay scheduling to work val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong @@ -101,11 +101,11 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) - case RemoteClientDisconnected(transport, address) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected")) + case DisassociatedEvent(_, remoteAddress, _) => + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected")) - case RemoteClientShutdown(transport, address) => - addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown")) + case AssociationErrorEvent(_, _, remoteAddress, _) => + addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown")) } // Make fake resource offers on all executors diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala index 3dbe61d706..babe875fa1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala @@ -119,7 +119,7 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), StandaloneSchedulerBackend.ACTOR_NAME) 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 e674d120ea..af1c36b34d 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -48,28 +48,27 @@ private[spark] object AkkaUtils { val akkaConf = ConfigFactory.parseString(""" akka.daemonic = on - akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] + akka.loggers = [""akka.event.slf4j.Slf4jLogger""] akka.stdout-loglevel = "ERROR" akka.actor.provider = "akka.remote.RemoteActorRefProvider" - akka.remote.transport = "akka.remote.netty.NettyRemoteTransport" - akka.remote.netty.hostname = "%s" - akka.remote.netty.port = %d - akka.remote.netty.connection-timeout = %ds - akka.remote.netty.message-frame-size = %d MiB - akka.remote.netty.execution-pool-size = %d + akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" + akka.remote.netty.tcp.hostname = "%s" + akka.remote.netty.tcp.port = %d + akka.remote.netty.tcp.connection-timeout = %ds + akka.remote.netty.tcp.message-frame-size = %d MiB + akka.remote.netty.tcp.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d akka.remote.log-remote-lifecycle-events = %s - akka.remote.netty.write-timeout = %ds - """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, - lifecycleEvents, akkaWriteTimeout)) + """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, + lifecycleEvents)) val actorSystem = ActorSystem(name, akkaConf) // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a // hack because Akka doesn't let you figure out the port through the public API yet. val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider - val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get - return (actorSystem, boundPort) + val boundPort = provider.getDefaultAddress.port.get + (actorSystem, boundPort) } } -- cgit v1.2.3 From 9865fd6aa07fd722c6782ec8001b732c39a98d45 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 30 Sep 2013 18:09:12 +0530 Subject: Fixed non termination of Executor backend, when sc.stop is not called. --- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 2 +- .../scala/org/apache/spark/executor/StandaloneExecutorBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 1f04c1eea5..3904b701b2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -165,7 +165,7 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case _: Terminated | DisassociatedEvent | AssociationErrorEvent => + case DisassociatedEvent(_, _, _) => masterDisconnected() case RequestWorkerState => { diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala index 46f0ef2cc6..f705a5631a 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -81,7 +81,7 @@ private[spark] class StandaloneExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case _: Terminated | DisassociatedEvent | AssociationErrorEvent => + case DisassociatedEvent(_, _, _) => logError("Driver terminated or disconnected! Shutting down.") System.exit(1) } -- cgit v1.2.3 From e09f4a9601b18921c309903737d309eab5c6d891 Mon Sep 17 00:00:00 2001 From: Martin Weindel Date: Sat, 5 Oct 2013 23:08:23 +0200 Subject: fixed some warnings --- .../org/apache/spark/network/netty/FileClient.java | 2 - .../org/apache/spark/network/netty/FileServer.java | 1 - .../java/JavaSparkContextVarargsWorkaround.java | 1 - .../spark/api/java/function/DoubleFunction.java | 2 - .../apache/spark/api/java/function/Function.java | 1 - .../apache/spark/api/java/function/Function2.java | 1 - .../api/java/function/PairFlatMapFunction.java | 1 - .../spark/api/java/function/PairFunction.java | 1 - .../spark/api/python/PythonWorkerFactory.scala | 4 +- .../org/apache/spark/deploy/ExecutorState.scala | 2 +- .../spark/deploy/master/ApplicationState.scala | 2 +- .../apache/spark/deploy/master/WorkerState.scala | 2 +- .../main/scala/org/apache/spark/rdd/BlockRDD.scala | 3 +- .../scala/org/apache/spark/rdd/CoalescedRDD.scala | 3 +- .../main/scala/org/apache/spark/rdd/EmptyRDD.scala | 3 +- .../scala/org/apache/spark/rdd/FilteredRDD.scala | 3 +- .../scala/org/apache/spark/rdd/FlatMappedRDD.scala | 3 +- .../scala/org/apache/spark/rdd/GlommedRDD.scala | 3 +- .../org/apache/spark/rdd/MapPartitionsRDD.scala | 3 +- .../spark/rdd/MapPartitionsWithIndexRDD.scala | 3 +- .../scala/org/apache/spark/rdd/MappedRDD.scala | 3 +- .../org/apache/spark/rdd/OrderedRDDFunctions.scala | 7 +- .../org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 3 +- .../scheduler/cluster/ClusterTaskSetManager.scala | 2 +- .../spark/scheduler/cluster/SchedulingMode.scala | 2 +- .../spark/scheduler/cluster/TaskLocality.scala | 2 +- .../scala/org/apache/spark/DistributedSuite.scala | 2 +- .../scala/org/apache/spark/UnpersistSuite.scala | 2 +- .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 8 +- .../org/apache/spark/examples/JavaLogQuery.java | 2 +- .../org/apache/spark/examples/JavaPageRank.java | 3 +- .../org/apache/spark/examples/JavaWordCount.java | 2 +- .../org/apache/spark/mllib/examples/JavaALS.java | 1 - .../spark/mllib/clustering/JavaKMeansSuite.java | 4 +- .../spark/mllib/recommendation/JavaALSSuite.java | 2 - .../spark/streaming/dstream/CoGroupedDStream.scala | 3 +- .../streaming/dstream/ConstantInputDStream.scala | 3 +- .../spark/streaming/dstream/FilteredDStream.scala | 3 +- .../streaming/dstream/FlatMapValuedDStream.scala | 3 +- .../streaming/dstream/FlatMappedDStream.scala | 3 +- .../spark/streaming/dstream/ForEachDStream.scala | 3 +- .../spark/streaming/dstream/GlommedDStream.scala | 3 +- .../streaming/dstream/MapPartitionedDStream.scala | 3 +- .../spark/streaming/dstream/MapValuedDStream.scala | 3 +- .../spark/streaming/dstream/MappedDStream.scala | 3 +- .../streaming/dstream/PluggableInputDStream.scala | 3 +- .../streaming/dstream/QueueInputDStream.scala | 4 +- .../spark/streaming/dstream/ShuffledDStream.scala | 3 +- .../streaming/dstream/TransformedDStream.scala | 3 +- .../org/apache/spark/streaming/JavaAPISuite.java | 94 +++++++++++----------- 51 files changed, 120 insertions(+), 108 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java index 20a7a3aa8c..edd0fc56f8 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java @@ -19,8 +19,6 @@ package org.apache.spark.network.netty; import io.netty.bootstrap.Bootstrap; import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; import io.netty.channel.socket.oio.OioSocketChannel; diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java index 666432474d..a99af348ce 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java @@ -20,7 +20,6 @@ package org.apache.spark.network.netty; import java.net.InetSocketAddress; import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; import io.netty.channel.oio.OioEventLoopGroup; diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java index c9cbce5624..2090efd3b9 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -17,7 +17,6 @@ package org.apache.spark.api.java; -import java.util.Arrays; import java.util.ArrayList; import java.util.List; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java index db34cd190a..ed92d31af5 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.java @@ -18,8 +18,6 @@ package org.apache.spark.api.java.function; -import scala.runtime.AbstractFunction1; - import java.io.Serializable; /** diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java index f9dae6ed34..e97116986f 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java @@ -19,7 +19,6 @@ package org.apache.spark.api.java.function; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java index 1659bfc552..cf77bb6b73 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java @@ -19,7 +19,6 @@ package org.apache.spark.api.java.function; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction2; import java.io.Serializable; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java index 5a5c9b6296..fbd0cdabe0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java @@ -20,7 +20,6 @@ package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java index 4c39f483e5..f09559627d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java @@ -20,7 +20,6 @@ package org.apache.spark.api.java.function; import scala.Tuple2; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; -import scala.runtime.AbstractFunction1; import java.io.Serializable; diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 67d45723ba..f291266fcf 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -64,7 +64,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String startDaemon() new Socket(daemonHost, daemonPort) } - case e => throw e + case e: Throwable => throw e } } } @@ -198,7 +198,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } }.start() } catch { - case e => { + case e: Throwable => { stopDaemon() throw e } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index fcfea96ad6..72f84c99fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy private[spark] object ExecutorState - extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") { + extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index 7e804223cf..39ef090ddf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.master private[spark] object ApplicationState - extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") { + extends Enumeration { type ApplicationState = Value diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala index b5ee6dca79..fb3fe88d92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") { +private[spark] object WorkerState extends Enumeration { type WorkerState = Value val ALIVE, DEAD, DECOMMISSIONED = Value diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index bca6956a18..fe2946bcbe 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,13 +19,14 @@ package org.apache.spark.rdd import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.BlockManager +import scala.reflect.ClassTag private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition { val index = idx } private[spark] -class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String]) +class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[String]) extends RDD[T](sc, Nil) { @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index c5de6362a9..98da35763b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -22,6 +22,7 @@ import java.io.{ObjectOutputStream, IOException} import scala.collection.mutable import scala.Some import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag /** * Class that captures a coalesced RDD by essentially keeping track of parent partitions @@ -68,7 +69,7 @@ case class CoalescedRDDPartition( * @param maxPartitions number of desired partitions in the coalesced RDD * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ -class CoalescedRDD[T: ClassManifest]( +class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, balanceSlack: Double = 0.10) diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index c8900d1a93..99c34c6cc5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -18,12 +18,13 @@ package org.apache.spark.rdd import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} +import scala.reflect.ClassTag /** * An RDD that is empty, i.e. has no element in it. */ -class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) { +class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala index 5312dc0b59..e74c83b90b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala @@ -18,8 +18,9 @@ package org.apache.spark.rdd import org.apache.spark.{OneToOneDependency, Partition, TaskContext} +import scala.reflect.ClassTag -private[spark] class FilteredRDD[T: ClassManifest]( +private[spark] class FilteredRDD[T: ClassTag]( prev: RDD[T], f: T => Boolean) extends RDD[T](prev) { diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala index cbdf6d84c0..4d1878fc14 100644 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala @@ -18,10 +18,11 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag private[spark] -class FlatMappedRDD[U: ClassManifest, T: ClassManifest]( +class FlatMappedRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: T => TraversableOnce[U]) extends RDD[U](prev) { diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala index 829545d7b0..1a694475f6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala @@ -18,8 +18,9 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag -private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T]) +private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T]) extends RDD[Array[T]](prev) { override def getPartitions: Array[Partition] = firstParent[T].partitions diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala index 203179c4ea..cdb5946b49 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala @@ -18,10 +18,11 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag private[spark] -class MapPartitionsRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false) diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala index 3ed8339010..3cf22851dd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag /** @@ -26,7 +27,7 @@ import org.apache.spark.{Partition, TaskContext} * information such as the number of tuples in a partition. */ private[spark] -class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest]( +class MapPartitionsWithIndexRDD[U: ClassTag, T: ClassTag]( prev: RDD[T], f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala index e8be1c4816..eb3b19907d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -18,9 +18,10 @@ package org.apache.spark.rdd import org.apache.spark.{Partition, TaskContext} +import scala.reflect.ClassTag private[spark] -class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U) +class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) extends RDD[U](prev) { override def getPartitions: Array[Partition] = firstParent[T].partitions diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 697be8b997..4a465840c6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{RangePartitioner, Logging} +import scala.reflect.ClassTag /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through @@ -25,9 +26,9 @@ import org.apache.spark.{RangePartitioner, Logging} * use these functions. They will work with any key type that has a `scala.math.Ordered` * implementation. */ -class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, - V: ClassManifest, - P <: Product2[K, V] : ClassManifest]( +class OrderedRDDFunctions[K <% Ordered[K]: ClassTag, + V: ClassTag, + P <: Product2[K, V] : ClassTag]( self: RDD[P]) extends Logging with Serializable { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index aed585e6a1..c8e623081a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -51,7 +51,7 @@ import org.apache.spark.Partitioner.defaultPartitioner * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)]) +class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) extends Logging with SparkHadoopMapReduceUtil with Serializable { diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 9537152335..b7205865cf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext} +import scala.reflect.ClassTag private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -32,7 +33,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @tparam K the key class. * @tparam V the value class. */ -class ShuffledRDD[K, V, P <: Product2[K, V] : ClassManifest]( +class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) extends RDD[P](prev.context, Nil) { 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 0ac3d7bcfd..f61fde6957 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 @@ -461,7 +461,7 @@ private[spark] class ClusterTaskSetManager( case cnf: ClassNotFoundException => val loader = Thread.currentThread().getContextClassLoader throw new SparkException("ClassNotFound with classloader: " + loader, cnf) - case ex => throw ex + case ex: Throwable => throw ex } // Mark finished and stop if we've finished all the tasks finished(index) = true diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala index 34811389a0..16013b3208 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala @@ -22,7 +22,7 @@ package org.apache.spark.scheduler.cluster * to order tasks amongst a Schedulable's sub-queues * "NONE" is used when the a Schedulable has no sub-queues. */ -object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") { +object SchedulingMode extends Enumeration { type SchedulingMode = Value val FAIR,FIFO,NONE = Value diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala index 5d4130e14a..8d8d708612 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster private[spark] object TaskLocality - extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY") + extends Enumeration { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index c719a54a61..adc6ca94ff 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -313,7 +313,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter Thread.sleep(200) } } catch { - case _ => { Thread.sleep(10) } + case _: Throwable => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala index 46a2da1724..768ca3850e 100644 --- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala +++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala @@ -37,7 +37,7 @@ class UnpersistSuite extends FunSuite with LocalSparkContext { Thread.sleep(200) } } catch { - case _ => { Thread.sleep(10) } + case _: Throwable => { Thread.sleep(10) } // Do nothing. We might see exceptions because block manager // is racing this thread to remove entries from the driver. } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index c1df5e151e..096023f476 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -225,8 +225,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { // test that you get over 90% locality in each group val minLocality = coalesced2.partitions .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.)((perc, loc) => math.min(perc,loc)) - assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%") + .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) + assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%") // test that the groups are load balanced with 100 +/- 20 elements in each val maxImbalance = coalesced2.partitions @@ -238,9 +238,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { val coalesced3 = data3.coalesce(numMachines*2) val minLocality2 = coalesced3.partitions .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.)((perc, loc) => math.min(perc,loc)) + .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + - (minLocality2*100.).toInt + "%") + (minLocality2*100.0).toInt + "%") } test("zipped RDDs") { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 152f029213..407cd7ccfa 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -123,7 +123,7 @@ public class JavaLogQuery { }); List, Stats>> output = counts.collect(); - for (Tuple2 t : output) { + for (Tuple2 t : output) { System.out.println(t._1 + "\t" + t._2); } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index c5603a639b..89aed8f279 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -21,7 +21,6 @@ import scala.Tuple2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFlatMapFunction; @@ -106,7 +105,7 @@ public class JavaPageRank { // Collects all URL ranks and dump them to console. List> output = ranks.collect(); - for (Tuple2 tuple : output) { + for (Tuple2 tuple : output) { System.out.println(tuple._1 + " has rank: " + tuple._2 + "."); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 07d32ad659..bd6383e13d 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -58,7 +58,7 @@ public class JavaWordCount { }); List> output = counts.collect(); - for (Tuple2 tuple : output) { + for (Tuple2 tuple : output) { System.out.println(tuple._1 + ": " + tuple._2); } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 628cb892b6..45a0d237da 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -25,7 +25,6 @@ import org.apache.spark.mllib.recommendation.ALS; import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; import org.apache.spark.mllib.recommendation.Rating; -import java.io.Serializable; import java.util.Arrays; import java.util.StringTokenizer; diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 32d3934ac1..33b99f4bd3 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -77,7 +77,7 @@ public class JavaKMeansSuite implements Serializable { @Test public void runKMeansUsingStaticMethods() { - List points = new ArrayList(); + List points = new ArrayList(); points.add(new double[]{1.0, 2.0, 6.0}); points.add(new double[]{1.0, 3.0, 0.0}); points.add(new double[]{1.0, 4.0, 6.0}); @@ -94,7 +94,7 @@ public class JavaKMeansSuite implements Serializable { @Test public void runKMeansUsingConstructor() { - List points = new ArrayList(); + List points = new ArrayList(); points.add(new double[]{1.0, 2.0, 6.0}); points.add(new double[]{1.0, 3.0, 0.0}); points.add(new double[]{1.0, 4.0, 6.0}); diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index 3323f6cee2..c474e01188 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -20,8 +20,6 @@ package org.apache.spark.mllib.recommendation; import java.io.Serializable; import java.util.List; -import scala.Tuple2; - import org.junit.After; import org.junit.Assert; import org.junit.Before; diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala index 4eddc755b9..16c1567355 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala @@ -21,9 +21,10 @@ import org.apache.spark.Partitioner import org.apache.spark.rdd.RDD import org.apache.spark.rdd.CoGroupedRDD import org.apache.spark.streaming.{Time, DStream, Duration} +import scala.reflect.ClassTag private[streaming] -class CoGroupedDStream[K : ClassManifest]( +class CoGroupedDStream[K : ClassTag]( parents: Seq[DStream[(K, _)]], partitioner: Partitioner ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala index a9a05c9981..f396c34758 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala @@ -19,11 +19,12 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Time, StreamingContext} +import scala.reflect.ClassTag /** * An input stream that always returns the same RDD on each timestep. Useful for testing. */ -class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T]) +class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T]) extends InputDStream[T](ssc_) { override def start() {} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala index 91ee2c1a36..db2e0a4cee 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class FilteredDStream[T: ClassManifest]( +class FilteredDStream[T: ClassTag]( parent: DStream[T], filterFunc: T => Boolean ) extends DStream[T](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala index ca7d7ca49e..244dc3ee4f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala @@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import scala.reflect.ClassTag private[streaming] -class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( +class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], flatMapValueFunc: V => TraversableOnce[U] ) extends DStream[(K, U)](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala index b37966f9a7..336c4b7a92 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class FlatMappedDStream[T: ClassManifest, U: ClassManifest]( +class FlatMappedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], flatMapFunc: T => Traversable[U] ) extends DStream[U](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index e21bac4602..98b14cb224 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Job, Time} +import scala.reflect.ClassTag private[streaming] -class ForEachDStream[T: ClassManifest] ( +class ForEachDStream[T: ClassTag] ( parent: DStream[T], foreachFunc: (RDD[T], Time) => Unit ) extends DStream[Unit](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala index 4294b07d91..23136f44fa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class GlommedDStream[T: ClassManifest](parent: DStream[T]) +class GlommedDStream[T: ClassTag](parent: DStream[T]) extends DStream[Array[T]](parent.ssc) { override def dependencies = List(parent) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala index 5329601a6f..8a04060e5b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class MapPartitionedDStream[T: ClassManifest, U: ClassManifest]( +class MapPartitionedDStream[T: ClassTag, U: ClassTag]( parent: DStream[T], mapPartFunc: Iterator[T] => Iterator[U], preservePartitioning: Boolean diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala index 8290df90a2..0ce364fd46 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala @@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import scala.reflect.ClassTag private[streaming] -class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest]( +class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag]( parent: DStream[(K, V)], mapValueFunc: V => U ) extends DStream[(K, U)](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala index b1682afea3..c0b7491d09 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.{Duration, DStream, Time} import org.apache.spark.rdd.RDD +import scala.reflect.ClassTag private[streaming] -class MappedDStream[T: ClassManifest, U: ClassManifest] ( +class MappedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], mapFunc: T => U ) extends DStream[U](parent.ssc) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 15782f5c11..6f9477020a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -18,9 +18,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.StreamingContext +import scala.reflect.ClassTag private[streaming] -class PluggableInputDStream[T: ClassManifest]( +class PluggableInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index 7d9f3521b1..97325f8ea3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -19,13 +19,13 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD - import scala.collection.mutable.Queue import scala.collection.mutable.ArrayBuffer import org.apache.spark.streaming.{Time, StreamingContext} +import scala.reflect.ClassTag private[streaming] -class QueueInputDStream[T: ClassManifest]( +class QueueInputDStream[T: ClassTag]( @transient ssc: StreamingContext, val queue: Queue[RDD[T]], oneAtATime: Boolean, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala index a95e66d761..e6e0022097 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala @@ -21,9 +21,10 @@ import org.apache.spark.Partitioner import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Duration, DStream, Time} +import scala.reflect.ClassTag private[streaming] -class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest]( +class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag]( parent: DStream[(K,V)], createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 60485adef9..73e1ddf7a4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Duration, DStream, Time} +import scala.reflect.ClassTag private[streaming] -class TransformedDStream[T: ClassManifest, U: ClassManifest] ( +class TransformedDStream[T: ClassTag, U: ClassTag] ( parent: DStream[T], transformFunc: (RDD[T], Time) => RDD[U] ) extends DStream[U](parent.ssc) { diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 783b8dea31..076fb53fa1 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -21,34 +21,36 @@ import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.io.Files; + import kafka.serializer.StringDecoder; + import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; + import scala.Tuple2; +import twitter4j.Status; + import org.apache.spark.HashPartitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaRDDLike; -import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.dstream.SparkFlumeEvent; import org.apache.spark.streaming.JavaTestUtils; import org.apache.spark.streaming.JavaCheckpointTestUtils; -import org.apache.spark.streaming.InputStreamsSuite; import java.io.*; import java.util.*; import akka.actor.Props; import akka.zeromq.Subscribe; -import akka.util.ByteString; // The test suite itself is Serializable so that anonymous Function implementations can be @@ -85,8 +87,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(3L), Arrays.asList(1L)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream count = stream.count(); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream count = stream.count(); JavaTestUtils.attachTestOutputStream(count); List> result = JavaTestUtils.runStreams(ssc, 3, 3); assertOrderInvariantEquals(expected, result); @@ -102,8 +104,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(5,5), Arrays.asList(9,4)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(new Function() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(new Function() { @Override public Integer call(String s) throws Exception { return s.length(); @@ -128,8 +130,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(7,8,9,4,5,6), Arrays.asList(7,8,9)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream windowed = stream.window(new Duration(2000)); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(2000)); JavaTestUtils.attachTestOutputStream(windowed); List> result = JavaTestUtils.runStreams(ssc, 4, 4); @@ -152,8 +154,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18), Arrays.asList(13,14,15,16,17,18)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000)); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000)); JavaTestUtils.attachTestOutputStream(windowed); List> result = JavaTestUtils.runStreams(ssc, 8, 4); @@ -170,8 +172,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList("giants"), Arrays.asList("yankees")); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream filtered = stream.filter(new Function() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream filtered = stream.filter(new Function() { @Override public Boolean call(String s) throws Exception { return s.contains("a"); @@ -193,8 +195,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(Arrays.asList("giants", "dodgers")), Arrays.asList(Arrays.asList("yankees", "red socks"))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream glommed = stream.glom(); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream> glommed = stream.glom(); JavaTestUtils.attachTestOutputStream(glommed); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -211,8 +213,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList("GIANTSDODGERS"), Arrays.asList("YANKEESRED SOCKS")); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream mapped = stream.mapPartitions(new FlatMapFunction, String>() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream mapped = stream.mapPartitions(new FlatMapFunction, String>() { @Override public Iterable call(Iterator in) { String out = ""; @@ -254,8 +256,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(15), Arrays.asList(24)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reduced = stream.reduce(new IntegerSum()); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reduced = stream.reduce(new IntegerSum()); JavaTestUtils.attachTestOutputStream(reduced); List> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -275,8 +277,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList(39), Arrays.asList(24)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(), + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(), new IntegerDifference(), new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(reducedWindowed); List> result = JavaTestUtils.runStreams(ssc, 4, 4); @@ -349,8 +351,8 @@ public class JavaAPISuite implements Serializable { Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"), Arrays.asList("a","t","h","l","e","t","i","c","s")); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream flatMapped = stream.flatMap(new FlatMapFunction() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream flatMapped = stream.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { return Lists.newArrayList(x.split("(?!^)")); @@ -396,8 +398,8 @@ public class JavaAPISuite implements Serializable { new Tuple2(9, "c"), new Tuple2(9, "s"))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { @Override public Iterable> call(String in) throws Exception { List> out = Lists.newArrayList(); @@ -430,10 +432,10 @@ public class JavaAPISuite implements Serializable { Arrays.asList(2,2,5,5), Arrays.asList(3,3,6,6)); - JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2); - JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2); + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2); - JavaDStream unioned = stream1.union(stream2); + JavaDStream unioned = stream1.union(stream2); JavaTestUtils.attachTestOutputStream(unioned); List> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -444,7 +446,7 @@ public class JavaAPISuite implements Serializable { * Performs an order-invariant comparison of lists representing two RDD streams. This allows * us to account for ordering variation within individual RDD's which occurs during windowing. */ - public static void assertOrderInvariantEquals( + public static > void assertOrderInvariantEquals( List> expected, List> actual) { for (List list: expected) { Collections.sort(list); @@ -467,11 +469,11 @@ public class JavaAPISuite implements Serializable { Arrays.asList(new Tuple2("giants", 6)), Arrays.asList(new Tuple2("yankees", 7))); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = stream.map( new PairFunction() { @Override - public Tuple2 call(String in) throws Exception { + public Tuple2 call(String in) throws Exception { return new Tuple2(in, in.length()); } }); @@ -1163,8 +1165,8 @@ public class JavaAPISuite implements Serializable { File tempDir = Files.createTempDir(); ssc.checkpoint(tempDir.getAbsolutePath()); - JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream letterCount = stream.map(new Function() { + JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(new Function() { @Override public Integer call(String s) throws Exception { return s.length(); @@ -1220,20 +1222,20 @@ public class JavaAPISuite implements Serializable { @Test public void testKafkaStream() { HashMap topics = Maps.newHashMap(); - JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); - JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, + JavaDStream test1 = ssc.kafkaStream("localhost:12345", "group", topics); + JavaDStream test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zk.connect","localhost:12345"); kafkaParams.put("groupid","consumer-group"); - JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, + JavaDStream test3 = ssc.kafkaStream(String.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK()); } @Test public void testSocketTextStream() { - JavaDStream test = ssc.socketTextStream("localhost", 12345); + JavaDStream test = ssc.socketTextStream("localhost", 12345); } @Test @@ -1253,7 +1255,7 @@ public class JavaAPISuite implements Serializable { } } - JavaDStream test = ssc.socketStream( + JavaDStream test = ssc.socketStream( "localhost", 12345, new Converter(), @@ -1262,39 +1264,39 @@ public class JavaAPISuite implements Serializable { @Test public void testTextFileStream() { - JavaDStream test = ssc.textFileStream("/tmp/foo"); + JavaDStream test = ssc.textFileStream("/tmp/foo"); } @Test public void testRawSocketStream() { - JavaDStream test = ssc.rawSocketStream("localhost", 12345); + JavaDStream test = ssc.rawSocketStream("localhost", 12345); } @Test public void testFlumeStream() { - JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY()); } @Test public void testFileStream() { JavaPairDStream foo = - ssc.fileStream("/tmp/foo"); + ssc.>fileStream("/tmp/foo"); } @Test public void testTwitterStream() { String[] filters = new String[] { "good", "bad", "ugly" }; - JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY()); } @Test public void testActorStream() { - JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY()); + JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY()); } @Test public void testZeroMQStream() { - JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function>() { + JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function>() { @Override public Iterable call(byte[][] b) throws Exception { return null; -- cgit v1.2.3 From 34da58ae50cd97a1136d45484130addcf6ac8a33 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 10 Oct 2013 14:03:56 +0530 Subject: Changed message-frame-size to maximum-frame-size as property. Removed a test accidentally added during merge. --- .../src/main/scala/org/apache/spark/executor/Executor.scala | 2 +- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 4 ++-- core/src/test/scala/org/apache/spark/DistributedSuite.scala | 13 ------------- .../spark/scheduler/cluster/TaskResultGetterSuite.scala | 4 ++-- 4 files changed, 5 insertions(+), 18 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 b4153f3533..3800063234 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -105,7 +105,7 @@ private[spark] class Executor( SparkEnv.set(env) env.metricsSystem.registerSource(executorSource) - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size") + private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") // Start worker thread pool val threadPool = new ThreadPoolExecutor( 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 af1c36b34d..8daf50ab69 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -54,8 +54,8 @@ private[spark] object AkkaUtils { akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" akka.remote.netty.tcp.hostname = "%s" akka.remote.netty.tcp.port = %d - akka.remote.netty.tcp.connection-timeout = %ds - akka.remote.netty.tcp.message-frame-size = %d MiB + akka.remote.netty.tcp.connection-timeout = %d s + akka.remote.netty.tcp.maximum-frame-size = %dMiB akka.remote.netty.tcp.execution-pool-size = %d akka.actor.default-dispatcher.throughput = %d akka.remote.log-remote-lifecycle-events = %s diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 25b9c3eb78..988ab1747d 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -320,19 +320,6 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter } } - test("job should fail if TaskResult exceeds Akka frame size") { - // We must use local-cluster mode since results are returned differently - // when running under LocalScheduler: - sc = new SparkContext("local-cluster[1,1,512]", "test") - val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt - val rdd = sc.parallelize(Seq(1)).map{x => new Array[Byte](akkaFrameSize)} - val exception = intercept[SparkException] { - rdd.reduce((x, y) => x) - } - exception.getMessage should endWith("result exceeded Akka frame size") - } - } object DistributedSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala index 370a3eb0eb..a00198db8c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala @@ -81,7 +81,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("handling results larger than Akka frame size") { val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) assert(result === 1.to(akkaFrameSize).toArray) @@ -102,7 +102,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA } scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) val akkaFrameSize = - sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.message-frame-size").toInt + sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) assert(result === 1.to(akkaFrameSize).toArray) -- cgit v1.2.3 From bfbd7e5d9f9e556961e79135dcde8f88fa31e2bc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 10 Oct 2013 15:22:31 +0530 Subject: Fixed some scala warnings in core. --- core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 2 +- .../scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 d7b45d4caa..c9c13f7a26 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 @@ -72,7 +72,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[R: ClassManifest]( + def mapPartitionsWithIndex[R: ClassTag]( f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala index feec8ecfe4..b2a8f06472 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala @@ -87,7 +87,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex => + case ex: Throwable => taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex)) } } @@ -111,7 +111,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche val loader = Thread.currentThread.getContextClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex => {} + case ex: Throwable => {} } scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } -- cgit v1.2.3 From a8bfdd4377918bf665d4615c7b45ed84a7a9cebc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 12 Nov 2013 11:38:20 +0530 Subject: Enabled remote death watch and a way to configure the timeouts for akka heartbeats. --- .../spark/executor/StandaloneExecutorBackend.scala | 4 +- .../cluster/StandaloneSchedulerBackend.scala | 5 --- .../scala/org/apache/spark/util/AkkaUtils.scala | 44 ++++++++++++---------- 3 files changed, 27 insertions(+), 26 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala index f705a5631a..a76a8e9730 100644 --- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala @@ -59,12 +59,12 @@ private[spark] class StandaloneExecutorBackend( driver = context.actorSelection(driverUrl) driver ! RegisterExecutor(executorId, hostPort, cores) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - // context.watch(driver) // Doesn't work with remote actors, but useful for testing } override def receive = { case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") + context.watch(sender) //Start watching for terminated messages. // Make this host instead of hostPort ? executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) @@ -81,7 +81,7 @@ private[spark] class StandaloneExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case DisassociatedEvent(_, _, _) => + case Terminated(actor) => logError("Driver terminated or disconnected! Shutting down.") System.exit(1) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index b6f0ec961a..2d09b329f2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -102,11 +102,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) - case DisassociatedEvent(_, remoteAddress, _) => - addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client disconnected")) - - case AssociationErrorEvent(_, _, remoteAddress, _) => - addressToExecutorId.get(remoteAddress).foreach(removeExecutor(_, "remote Akka client shutdown")) } // Make fake resource offers on all executors 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 8daf50ab69..2a831382df 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -43,29 +43,35 @@ private[spark] object AkkaUtils { val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - // 10 seconds is the default akka timeout, but in a cluster, we need higher by default. - val akkaWriteTimeout = System.getProperty("spark.akka.writeTimeout", "30").toInt - val akkaConf = ConfigFactory.parseString(""" - akka.daemonic = on - akka.loggers = [""akka.event.slf4j.Slf4jLogger""] - akka.stdout-loglevel = "ERROR" - akka.actor.provider = "akka.remote.RemoteActorRefProvider" - akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" - akka.remote.netty.tcp.hostname = "%s" - akka.remote.netty.tcp.port = %d - akka.remote.netty.tcp.connection-timeout = %d s - akka.remote.netty.tcp.maximum-frame-size = %dMiB - akka.remote.netty.tcp.execution-pool-size = %d - akka.actor.default-dispatcher.throughput = %d - akka.remote.log-remote-lifecycle-events = %s - """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize, - lifecycleEvents)) + val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "30").toInt + val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "30").toInt + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt + + val akkaConf = ConfigFactory.parseString( + s""" + |akka.daemonic = on + |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] + |akka.stdout-loglevel = "ERROR" + |akka.remote.watch-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.watch-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s + |akka.remote.watch-failure-detector.threshold = $akkaFailureDetector + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector + |akka.actor.provider = "akka.remote.RemoteActorRefProvider" + |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" + |akka.remote.netty.tcp.hostname = "$host" + |akka.remote.netty.tcp.port = $port + |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s + |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB + |akka.remote.netty.tcp.execution-pool-size = $akkaThreads + |akka.actor.default-dispatcher.throughput = $akkaBatchSize + |akka.remote.log-remote-lifecycle-events = $lifecycleEvents + """.stripMargin) val actorSystem = ActorSystem(name, akkaConf) - // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a - // hack because Akka doesn't let you figure out the port through the public API yet. val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get (actorSystem, boundPort) -- cgit v1.2.3 From 6860b79f6e4cc0d38b08848f19127c259d9b5069 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 12 Nov 2013 12:43:53 +0530 Subject: Remove deprecated actorFor and use actorSelection everywhere. --- .../scala/org/apache/spark/MapOutputTracker.scala | 8 +++++-- .../src/main/scala/org/apache/spark/SparkEnv.scala | 8 +++---- .../org/apache/spark/deploy/client/Client.scala | 23 +++++--------------- .../org/apache/spark/deploy/worker/Worker.scala | 23 ++++++++++++++------ .../apache/spark/storage/BlockManagerMaster.scala | 25 +++++++++++++--------- .../org/apache/spark/storage/ThreadingTest.scala | 2 +- .../org/apache/spark/MapOutputTrackerSuite.scala | 14 ++++++------ .../apache/spark/storage/BlockManagerSuite.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../streaming/dstream/NetworkInputDStream.scala | 2 +- 10 files changed, 58 insertions(+), 51 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1afb1870f1..6590e9779e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -60,7 +60,7 @@ private[spark] class MapOutputTracker extends Logging { private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds") // Set to the MapOutputTrackerActor living on the driver - var trackerActor: ActorRef = _ + var trackerActor: Either[ActorRef, ActorSelection] = _ private var mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] @@ -79,7 +79,11 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. def askTracker(message: Any): Any = { try { - val future = trackerActor.ask(message)(timeout) + val future = if (trackerActor.isLeft ) { + trackerActor.left.get.ask(message)(timeout) + } else { + trackerActor.right.get.ask(message)(timeout) + } return Await.result(future, timeout) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index a267407c67..0d9bd500e4 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -20,7 +20,7 @@ package org.apache.spark import collection.mutable import serializer.Serializer -import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem} +import akka.actor._ import akka.remote.RemoteActorRefProvider import org.apache.spark.broadcast.BroadcastManager @@ -161,17 +161,17 @@ object SparkEnv extends Logging { val closureSerializer = serializerManager.get( System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer")) - def registerOrLookup(name: String, newActor: => Actor): ActorRef = { + def registerOrLookup(name: String, newActor: => Actor): Either[ActorRef, ActorSelection] = { if (isDriver) { logInfo("Registering " + name) - actorSystem.actorOf(Props(newActor), name = name) + Left(actorSystem.actorOf(Props(newActor), name = name)) } else { val driverHost: String = System.getProperty("spark.driver.host", "localhost") val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt Utils.checkHost(driverHost, "Expected hostname") val url = "akka.tcp://spark@%s:%s/user/%s".format(driverHost, driverPort, name) logInfo("Connecting to " + name + ": " + url) - actorSystem.actorFor(url) + Right(actorSystem.actorSelection(url)) } } 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 164386782c..000d1ee9f8 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 @@ -49,18 +49,14 @@ private[spark] class Client( var appId: String = null class ClientActor extends Actor with Logging { - var master: ActorRef = null - var masterAddress: Address = null + var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times override def preStart() { logInfo("Connecting to master " + masterUrl) try { - master = context.actorFor(Master.toAkkaUrl(masterUrl)) - masterAddress = master.path.address + master = context.actorSelection(Master.toAkkaUrl(masterUrl)) master ! RegisterApplication(appDescription) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing } catch { case e: Exception => logError("Failed to connect to master", e) @@ -71,6 +67,7 @@ private[spark] class Client( override def receive = { case RegisteredApplication(appId_) => + context.watch(sender) appId = appId_ listener.connected(appId) @@ -92,18 +89,8 @@ private[spark] class Client( listener.executorRemoved(fullId, message.getOrElse(""), exitStatus) } - case Terminated(actor_) if actor_ == master => - logError("Connection to master failed; stopping client") - markDisconnected() - context.stop(self) - - case DisassociatedEvent(_, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") - markDisconnected() - context.stop(self) - - case AssociationErrorEvent(_, _, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + case Terminated(actor_) => + logError(s"Connection to $actor_ dropped, stopping client") markDisconnected() context.stop(self) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 3904b701b2..400d6f26ea 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -24,7 +24,7 @@ import java.io.File import scala.collection.mutable.HashMap import scala.concurrent.duration._ -import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated} +import akka.actor._ import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} import org.apache.spark.Logging @@ -34,6 +34,16 @@ import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.deploy.DeployMessages.WorkerStateResponse +import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed +import org.apache.spark.deploy.DeployMessages.KillExecutor +import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged +import scala.Some +import org.apache.spark.deploy.DeployMessages.Heartbeat +import org.apache.spark.deploy.DeployMessages.RegisteredWorker +import akka.remote.DisassociatedEvent +import org.apache.spark.deploy.DeployMessages.LaunchExecutor +import org.apache.spark.deploy.DeployMessages.RegisterWorker private[spark] class Worker( @@ -54,7 +64,7 @@ private[spark] class Worker( // Send a heartbeat every (heartbeat timeout) / 4 milliseconds val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4 - var master: ActorRef = null + var master: ActorSelection = null var masterWebUiUrl : String = "" val workerId = generateWorkerId() var sparkHome: File = null @@ -111,10 +121,8 @@ private[spark] class Worker( def connectToMaster() { logInfo("Connecting to master " + masterUrl) - master = context.actorFor(Master.toAkkaUrl(masterUrl)) + master = context.actorSelection(Master.toAkkaUrl(masterUrl)) master ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing } import context.dispatcher @@ -123,6 +131,8 @@ private[spark] class Worker( case RegisteredWorker(url) => masterWebUiUrl = url logInfo("Successfully registered with master") + context.watch(sender) // remote death watch for master + //TODO: Is heartbeat really necessary akka does it anyway ! context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) { master ! Heartbeat(workerId) } @@ -165,7 +175,8 @@ private[spark] class Worker( logInfo("Asked to kill unknown executor " + fullId) } - case DisassociatedEvent(_, _, _) => + case Terminated(actor_) => + logInfo(s"$actor_ terminated !") masterDisconnected() case RequestWorkerState => { 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 0c977f05d1..c1aa43d59c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -17,14 +17,7 @@ package org.apache.spark.storage -import java.io._ -import java.util.{HashMap => JHashMap} - -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import scala.util.Random - -import akka.actor.{Actor, ActorRef, ActorSystem, Props} +import akka.actor._ import scala.concurrent.Await import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global @@ -34,8 +27,16 @@ import scala.concurrent.duration._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.storage.BlockManagerMessages.GetLocations +import org.apache.spark.storage.BlockManagerMessages.GetLocationsMultipleBlockIds +import org.apache.spark.storage.BlockManagerMessages.RegisterBlockManager +import org.apache.spark.storage.BlockManagerMessages.HeartBeat +import org.apache.spark.storage.BlockManagerMessages.RemoveExecutor +import org.apache.spark.storage.BlockManagerMessages.GetPeers +import org.apache.spark.storage.BlockManagerMessages.RemoveBlock +import org.apache.spark.storage.BlockManagerMessages.RemoveRdd -private[spark] class BlockManagerMaster(var driverActor: ActorRef) 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 @@ -165,7 +166,11 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { - val future = driverActor.ask(message)(timeout) + val future = if (driverActor.isLeft ) { + driverActor.left.get.ask(message)(timeout) + } else { + driverActor.right.get.ask(message)(timeout) + } val result = Await.result(future, timeout) if (result == null) { throw new SparkException("BlockManagerMaster returned null") diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index f2ae8dd97d..1e6da269f2 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -93,7 +93,7 @@ private[spark] object ThreadingTest { val actorSystem = ActorSystem("test") val serializer = new KryoSerializer val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 18fb1bf590..955f6cdadc 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -49,14 +49,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -75,7 +75,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTracker() - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker))) + tracker.trackerActor = Left(actorSystem.actorOf(Props(new MapOutputTrackerActor(tracker)))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -103,13 +103,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { System.setProperty("spark.hostPort", hostname + ":" + boundPort) val masterTracker = new MapOutputTracker() - masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker") + masterTracker.trackerActor = Left(actorSystem.actorOf( + Props(new MapOutputTrackerActor(masterTracker)), "MapOutputTracker")) val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0) val slaveTracker = new MapOutputTracker() - slaveTracker.trackerActor = slaveSystem.actorFor( - "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker") + slaveTracker.trackerActor = Right(slaveSystem.actorSelection( + "akka.tcp://spark@localhost:" + boundPort + "/user/MapOutputTracker")) masterTracker.registerShuffle(10, 1) masterTracker.incrementEpoch() diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 038a9acb85..4fdc43cc22 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -53,7 +53,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT System.setProperty("spark.hostPort", "localhost:" + boundPort) master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true)))) + Left(actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 08e399f9ee..128711aacd 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -86,7 +86,7 @@ class FeederActor extends Actor { class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) extends Actor with Receiver { - lazy private val remotePublisher = context.actorFor(urlOfPublisher) + lazy private val remotePublisher = context.actorSelection(urlOfPublisher) override def preStart = remotePublisher ! SubscribeReceiver(context.self) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 394a39fbb0..b2f9f8b224 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -178,7 +178,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging val ip = System.getProperty("spark.driver.host", "localhost") val port = System.getProperty("spark.driver.port", "7077").toInt val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - val tracker = env.actorSystem.actorFor(url) + val tracker = env.actorSystem.actorSelection(url) val timeout = 5.seconds override def preStart() { -- cgit v1.2.3 From d4cd32330e1e4ac83b38bc922a9d3fd85f85f606 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Thu, 14 Nov 2013 13:57:02 +0800 Subject: Some fixes for previous master merge commits --- core/src/main/scala/org/apache/spark/deploy/client/Client.scala | 2 +- core/src/main/scala/org/apache/spark/deploy/master/Master.scala | 3 ++- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 4 +--- project/SparkBuild.scala | 1 + 4 files changed, 5 insertions(+), 5 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 be8693ec54..3953a3e178 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 @@ -21,7 +21,6 @@ import java.util.concurrent.TimeoutException import scala.concurrent.duration._ import scala.concurrent.Await -import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.actor.Terminated @@ -84,6 +83,7 @@ private[spark] class Client( def registerWithMaster() { tryRegisterAllMasters() + import context.dispatcher var retries = 0 lazy val retryTimer: Cancellable = context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { 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 26f980760d..0545ad185f 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 @@ -24,7 +24,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ import scala.concurrent.duration.{ Duration, FiniteDuration } -import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.pattern.ask @@ -58,6 +57,8 @@ import java.util.concurrent.TimeUnit private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { + import context.dispatcher + val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000 val RETAINED_APPLICATIONS = System.getProperty("spark.deploy.retainedApplications", "200").toInt diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 991b22d9f8..98c57ca0b0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -23,7 +23,6 @@ import java.io.File import scala.collection.mutable.HashMap import scala.concurrent.duration._ -import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} @@ -61,6 +60,7 @@ private[spark] class Worker( masterUrls: Array[String], workDirPath: String = null) extends Actor with Logging { + import context.dispatcher Utils.checkHost(host, "Expected hostname") assert (port > 0) @@ -175,8 +175,6 @@ private[spark] class Worker( retryTimer // start timer } - import context.dispatcher - override def receive = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9a3cbbe7d2..26e6a8326c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -227,6 +227,7 @@ object SparkBuild extends Build { "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), "org.apache.avro" % "avro" % "1.7.4", "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty), + "org.apache.zookeeper" % "zookeeper" % "3.4.5" excludeAll(excludeNetty), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", -- cgit v1.2.3 From f629ba95b6a1a3508463bfdcb03efcfaa3327cb5 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 14 Nov 2013 22:13:09 -0800 Subject: Various merge corrections I've diff'd this patch against my own -- since they were both created independently, this means that two sets of eyes have gone over all the merge conflicts that were created, so I'm feeling significantly more confident in the resulting PR. @rxin has looked at the changes to the repl and is resoundingly confident that they are correct. --- .../apache/spark/api/java/function/Function.java | 2 - .../apache/spark/api/java/function/Function2.java | 2 - .../org/apache/spark/deploy/client/Client.scala | 4 +- .../spark/deploy/master/ApplicationState.scala | 3 +- .../org/apache/spark/deploy/master/Master.scala | 14 +---- .../org/apache/spark/deploy/worker/Worker.scala | 17 +------ .../executor/CoarseGrainedExecutorBackend.scala | 1 - .../scala/org/apache/spark/executor/Executor.scala | 6 ++- .../org/apache/spark/rdd/AsyncRDDActions.scala | 2 +- .../main/scala/org/apache/spark/rdd/BlockRDD.scala | 3 +- .../scala/org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 7 +-- .../spark/scheduler/cluster/ClusterScheduler.scala | 2 +- .../mesos/CoarseMesosSchedulerBackend.scala | 1 + .../main/scala/org/apache/spark/util/Utils.scala | 5 +- .../apache/spark/util/collection/OpenHashMap.scala | 1 - docs/hadoop-third-party-distributions.md | 4 +- project/SparkBuild.scala | 11 +--- .../scala/org/apache/spark/repl/SparkILoop.scala | 14 ++--- .../scala/org/apache/spark/repl/ReplSuite.scala | 2 +- .../spark/streaming/NetworkInputTracker.scala | 2 +- .../spark/streaming/PairDStreamFunctions.scala | 1 - .../spark/streaming/api/java/JavaPairDStream.scala | 2 +- .../spark/streaming/dstream/CoGroupedDStream.scala | 59 ---------------------- .../streaming/dstream/KafkaInputDStream.scala | 1 - 25 files changed, 33 insertions(+), 135 deletions(-) delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java index 49e661a376..537439ef53 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java @@ -29,8 +29,6 @@ import java.io.Serializable; * when mapping RDDs of other types. */ public abstract class Function extends WrappedFunction1 implements Serializable { - public abstract R call(T t) throws Exception; - public ClassTag returnType() { return ClassTag$.MODULE$.apply(Object.class); } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java index cf77bb6b73..a2d1214fb4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java @@ -28,8 +28,6 @@ import java.io.Serializable; public abstract class Function2 extends WrappedFunction2 implements Serializable { - public abstract R call(T1 t1, T2 t2) throws Exception; - public ClassTag returnType() { return (ClassTag) ClassTag$.MODULE$.apply(Object.class); } 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 3953a3e178..572fc347df 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 @@ -145,11 +145,11 @@ private[spark] class Client( markDisconnected() case DisassociatedEvent(_, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case AssociationErrorEvent(_, _, address, _) if address == masterAddress => - logError("Connection to master failed; stopping client") + logWarning("Connection to master failed; waiting for master to reconnect...") markDisconnected() case StopClient => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala index a74d7be4c9..67e6c5d66a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala @@ -17,8 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object ApplicationState - extends Enumeration { +private[spark] object ApplicationState extends Enumeration { type ApplicationState = Value 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 0545ad185f..7db5097c2d 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 @@ -23,7 +23,7 @@ import java.text.SimpleDateFormat import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ -import scala.concurrent.duration.{ Duration, FiniteDuration } +import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor._ import akka.pattern.ask @@ -41,16 +41,6 @@ import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed import org.apache.spark.deploy.DeployMessages.KillExecutor import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import scala.Some -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisteredApplication -import org.apache.spark.deploy.DeployMessages.RegisterWorker -import org.apache.spark.deploy.DeployMessages.ExecutorUpdated -import org.apache.spark.deploy.DeployMessages.MasterStateResponse -import org.apache.spark.deploy.DeployMessages.ExecutorAdded -import org.apache.spark.deploy.DeployMessages.RegisterApplication -import org.apache.spark.deploy.DeployMessages.ApplicationRemoved -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker import akka.actor.Terminated import akka.serialization.SerializationExtension import java.util.concurrent.TimeUnit @@ -571,7 +561,7 @@ 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( + 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 diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 98c57ca0b0..07189ac850 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ -import akka.remote.{RemotingLifecycleEvent, AssociationErrorEvent, DisassociatedEvent} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.Logging import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} @@ -34,19 +34,6 @@ import org.apache.spark.deploy.master.Master import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{Utils, AkkaUtils} -import org.apache.spark.deploy.DeployMessages.WorkerStateResponse -import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed -import org.apache.spark.deploy.DeployMessages.KillExecutor -import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import scala.Some -import akka.remote.DisassociatedEvent -import org.apache.spark.deploy.DeployMessages.LaunchExecutor -import org.apache.spark.deploy.DeployMessages.RegisterWorker -import org.apache.spark.deploy.DeployMessages.WorkerSchedulerStateResponse -import org.apache.spark.deploy.DeployMessages.MasterChanged -import org.apache.spark.deploy.DeployMessages.Heartbeat -import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import akka.actor.Terminated /** * @param masterUrls Each url should look like spark://host:port. @@ -248,7 +235,7 @@ private[spark] class Worker( } } - case DisassociatedEvent(_, _, _) => + case DisassociatedEvent(_, address, _) if address == master.path.address => masterDisconnected() case RequestWorkerState => { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 73fa7d6b6a..50302fcca4 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -107,7 +107,6 @@ private[spark] object CoarseGrainedExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) - actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), name = "Executor") 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 de4540493a..0b0a60ee60 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -118,7 +118,11 @@ private[spark] class Executor( } } - private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") + // Akka's message frame size. If task result is bigger than this, we use the block manager + // to send the result back. + private val akkaFrameSize = { + env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") + } // Start worker thread pool val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 44c5078621..d1c74a5063 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -21,9 +21,9 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global +import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} -import scala.reflect.ClassTag /** * A set of asynchronous RDD actions available through an implicit conversion. diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 63b9fe1478..424354ae16 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd +import scala.reflect.ClassTag + import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import org.apache.spark.storage.{BlockId, BlockManager} -import scala.reflect.ClassTag private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 47e958b5e6..53f77a38f5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -52,7 +52,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * sources in HBase, or S3). * * @param sc The SparkContext to associate the RDD with. - * @param broadCastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed + * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. * Otherwise, a new JobConf will be created on each slave using the enclosed Configuration. * @param initLocalJobConfFuncOpt Optional closure used to initialize any JobConf that HadoopRDD diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 7b4fc6b9be..fdea3f6f88 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -20,13 +20,14 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.concurrent.duration._ -import scala.concurrent.ExecutionContext.Implicits.global -import akka.actor._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ import scala.reflect.ClassTag +import akka.actor._ + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 2d8a0a62c9..9975ec1ab6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -25,8 +25,8 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ import org.apache.spark._ import org.apache.spark.TaskState.TaskState diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 8de9b72b2f..84fe3094cc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -181,6 +181,7 @@ private[spark] class CoarseMesosSchedulerBackend( !slaveIdsWithExecutors.contains(slaveId)) { // Launch an executor on the slave val cpusToUse = math.min(cpus, maxCores - totalCoresAcquired) + totalCoresAcquired += cpusToUse val taskId = newMesosTaskId() taskIdToSlaveId(taskId) = slaveId slaveIdsWithExecutors += slaveId diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 7557ddab19..02adcb41c6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,14 +22,11 @@ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address} import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} - -import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import scala.collection.Map +import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import scala.Some - import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 45849b3380..c26f23d500 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,7 +19,6 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag - /** * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index f706625fe9..b33af2cf24 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -25,8 +25,8 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.

CDH Releases

- - + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 26e6a8326c..476e7c5800 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -105,12 +105,6 @@ object SparkBuild extends Build { // also check the local Maven repository ~/.m2 resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))), - // Shared between both core and streaming. - resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"), - - // Shared between both examples and streaming. - resolvers ++= Seq("Mqtt Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/"), - // For Sonatype publishing resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"), @@ -292,11 +286,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy), - "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1" + "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1" exclude("com.sun.jdmk", "jmxtools") exclude("com.sun.jmx", "jmxri") - exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeNetty), + exclude("net.sf.jopt-simple", "jopt-simple"), "org.eclipse.paho" % "mqtt-client" % "0.4.0", "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty), diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 43e504c290..523fd1222d 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -940,17 +940,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, if (prop != null) prop else "local" } } - val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')) - .getOrElse(new Array[String](0)) - .map(new java.io.File(_).getAbsolutePath) - try { - sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) - } catch { - case e: Exception => - e.printStackTrace() - echo("Failed to create SparkContext, exiting...") - sys.exit(1) - } + val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) + sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars) + echo("Created spark context..") sparkContext } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 418c31e24b..c230a03298 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -78,7 +78,7 @@ class ReplSuite extends FunSuite { System.clearProperty("spark.hostPort") } - test ("simple foreach with accumulator") { + test("simple foreach with accumulator") { val output = runInterpreter("local", """ |val accum = sc.accumulator(0) |sc.parallelize(1 to 10).foreach(x => accum += x) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala index 66fe6e7870..6e9a781978 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala @@ -25,10 +25,10 @@ import org.apache.spark.SparkContext._ import scala.collection.mutable.HashMap import scala.collection.mutable.Queue +import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import scala.concurrent.duration._ import akka.dispatch._ import org.apache.spark.storage.BlockId diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala index ea5c165691..80af96c060 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala @@ -33,7 +33,6 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.conf.Configuration -import scala.Some class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) extends Serializable { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 3ba37bed4d..dfd6e27c3e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -728,7 +728,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } object JavaPairDStream { - implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) : JavaPairDStream[K, V] = { + implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) = { new JavaPairDStream[K, V](dstream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala deleted file mode 100644 index 16c1567355..0000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/CoGroupedDStream.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.streaming.dstream - -import org.apache.spark.Partitioner -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.CoGroupedRDD -import org.apache.spark.streaming.{Time, DStream, Duration} -import scala.reflect.ClassTag - -private[streaming] -class CoGroupedDStream[K : ClassTag]( - parents: Seq[DStream[(K, _)]], - partitioner: Partitioner - ) extends DStream[(K, Seq[Seq[_]])](parents.head.ssc) { - - if (parents.length == 0) { - throw new IllegalArgumentException("Empty array of parents") - } - - if (parents.map(_.ssc).distinct.size > 1) { - throw new IllegalArgumentException("Array of parents have different StreamingContexts") - } - - if (parents.map(_.slideDuration).distinct.size > 1) { - throw new IllegalArgumentException("Array of parents have different slide times") - } - - override def dependencies = parents.toList - - override def slideDuration: Duration = parents.head.slideDuration - - override def compute(validTime: Time): Option[RDD[(K, Seq[Seq[_]])]] = { - val part = partitioner - val rdds = parents.flatMap(_.getOrCompute(validTime)) - if (rdds.size > 0) { - val q = new CoGroupedRDD[K](rdds, part) - Some(q) - } else { - None - } - } - -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala index ec0096c85f..526f5564c7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala @@ -33,7 +33,6 @@ import org.I0Itec.zkclient._ import scala.collection.Map import scala.reflect.ClassTag - /** * Input stream that pulls messages from a Kafka Broker. * -- cgit v1.2.3 From 77929cfeed95905106f5b3891e8de1b1c312d119 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 22 Nov 2013 19:46:39 +0530 Subject: Fine tuning defaults for akka and restored tracking of dissassociated events, for they are delivered when a remote TCP socket is closed. Also made transport failure heartbeats larger interval for it is mostly not needed. As we are using remote death watch instead. --- .../org/apache/spark/deploy/master/Master.scala | 23 ++++++++++++++++------ .../org/apache/spark/deploy/worker/Worker.scala | 12 +++++++---- .../executor/CoarseGrainedExecutorBackend.scala | 11 +++++------ .../cluster/CoarseGrainedSchedulerBackend.scala | 3 +++ .../scala/org/apache/spark/util/AkkaUtils.scala | 12 ++++++----- 5 files changed, 40 insertions(+), 21 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 a7cfc256a9..25f5927128 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 @@ -17,8 +17,9 @@ package org.apache.spark.deploy.master -import java.util.Date import java.text.SimpleDateFormat +import java.util.concurrent.TimeUnit +import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await @@ -28,6 +29,7 @@ import scala.concurrent.duration.{Duration, FiniteDuration} import akka.actor._ import akka.pattern.ask import akka.remote._ +import akka.serialization.SerializationExtension import akka.util.Timeout import org.apache.spark.{Logging, SparkException} @@ -40,11 +42,6 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed import org.apache.spark.deploy.DeployMessages.KillExecutor import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import scala.Some -import akka.actor.Terminated -import akka.serialization.SerializationExtension -import java.util.concurrent.TimeUnit - private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher @@ -102,6 +99,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act override def preStart() { logInfo("Starting Spark master at " + masterUrl) // Listen for remote client disconnection events, since they don't go through Akka's watch() + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.start() masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) @@ -267,11 +265,20 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act case Terminated(actor) => { // The disconnected actor could've been either a worker or an app; remove whichever of // those we have an entry for in the corresponding actor hashmap + logInfo(s"$actor got terminated, removing it.") actorToWorker.get(actor).foreach(removeWorker) actorToApp.get(actor).foreach(finishApplication) if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } } + case DisassociatedEvent(_, address, _) => { + // The disconnected client could've been either a worker or an app; remove whichever it was + logInfo(s"$address got disassociated, removing it.") + addressToWorker.get(address).foreach(removeWorker) + addressToApp.get(address).foreach(finishApplication) + if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } + } + case RequestMasterState => { sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray, state) @@ -431,6 +438,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act exec.id, ExecutorState.LOST, Some("worker lost"), None) exec.application.removeExecutor(exec) } + context.stop(worker.actor) + context.unwatch(worker.actor) persistenceEngine.removeWorker(worker) } @@ -493,6 +502,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act app.driver ! ApplicationRemoved(state.toString) } persistenceEngine.removeApplication(app) + context.stop(app.driver) + context.unwatch(app.driver) schedule() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 9472c9a619..3a7d0b859b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -17,14 +17,16 @@ package org.apache.spark.deploy.worker +import java.io.File import java.text.SimpleDateFormat import java.util.Date -import java.io.File import scala.collection.mutable.HashMap import scala.concurrent.duration._ import akka.actor._ +import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent} + import org.apache.spark.Logging import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ @@ -36,10 +38,8 @@ import org.apache.spark.deploy.DeployMessages.WorkerStateResponse import org.apache.spark.deploy.DeployMessages.RegisterWorkerFailed import org.apache.spark.deploy.DeployMessages.KillExecutor import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged -import scala.Some import org.apache.spark.deploy.DeployMessages.Heartbeat import org.apache.spark.deploy.DeployMessages.RegisteredWorker -import akka.remote.DisassociatedEvent import org.apache.spark.deploy.DeployMessages.LaunchExecutor import org.apache.spark.deploy.DeployMessages.RegisterWorker @@ -124,7 +124,7 @@ private[spark] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) - + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) webUi.start() registerWithMaster() @@ -249,6 +249,10 @@ private[spark] class Worker( logInfo(s"$actor_ terminated !") masterDisconnected() + case x: DisassociatedEvent => + logInfo(s"$x Disassociated !") + masterDisconnected() + case RequestWorkerState => { sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, activeMasterUrl, cores, memory, diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index a98ec06be9..2818a775d0 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -26,11 +26,6 @@ import org.apache.spark.Logging import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{Utils, AkkaUtils} -import akka.remote.DisassociatedEvent -import akka.remote.AssociationErrorEvent -import akka.remote.DisassociatedEvent -import akka.actor.Terminated - private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, @@ -82,7 +77,11 @@ private[spark] class CoarseGrainedExecutorBackend( } case Terminated(actor) => - logError(s"Driver $actor terminated or disconnected! Shutting down.") + logError(s"Driver $actor terminated, Shutting down.") + System.exit(1) + + case x: DisassociatedEvent => + logError(s"Driver $x disassociated! Shutting down.") System.exit(1) case StopExecutor => 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 821c30a119..e316f6b41f 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 @@ -121,6 +121,9 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac case Terminated(actor) => actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) + case DisassociatedEvent(_, address, _) => + addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) + } // Make fake resource offers on all executors 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 2a831382df..90a5387b2b 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -44,9 +44,11 @@ private[spark] object AkkaUtils { val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "30").toInt - val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "30").toInt - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt + val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "60").toInt + val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble + // Since we have our own Heart Beat mechanism and TCP already tracks connections. + // Using this makes very little sense. So setting this to a relatively larger value suffices. + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt val akkaConf = ConfigFactory.parseString( s""" @@ -56,8 +58,8 @@ private[spark] object AkkaUtils { |akka.remote.watch-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.watch-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.watch-failure-detector.threshold = $akkaFailureDetector - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.heartbeat-interval = 30 s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = ${akkaHeartBeatPauses + 10} s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" -- cgit v1.2.3 From 489862a65766d30278c186d280c6286937c81155 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 25 Nov 2013 17:55:21 +0530 Subject: Remote death watch has a funny bug. https://gist.github.com/ScrapCodes/4805fd84906e40b7b03d --- .../scala/org/apache/spark/deploy/client/Client.scala | 10 ++++------ .../scala/org/apache/spark/deploy/master/Master.scala | 17 ----------------- .../scala/org/apache/spark/deploy/worker/Worker.scala | 9 --------- .../spark/executor/CoarseGrainedExecutorBackend.scala | 5 ----- .../cluster/CoarseGrainedSchedulerBackend.scala | 4 ---- .../main/scala/org/apache/spark/util/AkkaUtils.scala | 11 +++-------- spark-class | 1 + 7 files changed, 8 insertions(+), 49 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 070f10f729..408692ec9c 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,7 +23,6 @@ import scala.concurrent.duration._ import scala.concurrent.Await import akka.actor._ -import akka.actor.Terminated import akka.pattern.AskTimeoutException import akka.pattern.ask import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} @@ -62,6 +61,7 @@ private[spark] class Client( var alreadyDead = false // To avoid calling listener.dead() multiple times override def preStart() { + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) try { registerWithMaster() } catch { @@ -107,7 +107,6 @@ private[spark] class Client( override def receive = { case RegisteredApplication(appId_, masterUrl) => - context.watch(sender) prevMaster = sender appId = appId_ registered = true @@ -123,7 +122,7 @@ private[spark] class Client( val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) listener.executorAdded(fullId, workerId, hostPort, cores, memory) - + case ExecutorUpdated(id, state, message, exitStatus) => val fullId = appId + "/" + id val messageText = message.map(s => " (" + s + ")").getOrElse("") @@ -134,13 +133,12 @@ private[spark] class Client( case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) - context.unwatch(prevMaster) changeMaster(masterUrl) alreadyDisconnected = false sender ! MasterChangeAcknowledged(appId) - case Terminated(actor_) => - logWarning(s"Connection to $actor_ failed; waiting for master to reconnect...") + case DisassociatedEvent(_, address, _) => + logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() case StopClient => 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 25f5927128..81fb5c4e43 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 @@ -147,9 +147,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act RecoveryState.ALIVE else RecoveryState.RECOVERING - logInfo("I have been elected leader! New state: " + state) - if (state == RecoveryState.RECOVERING) { beginRecovery(storedApps, storedWorkers) context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() } @@ -171,7 +169,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else { val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) registerWorker(worker) - context.watch(sender) persistenceEngine.addWorker(worker) sender ! RegisteredWorker(masterUrl, masterWebUiUrl) schedule() @@ -186,7 +183,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act val app = createApplication(description, sender) registerApplication(app) logInfo("Registered app " + description.name + " with ID " + app.id) - context.watch(sender) persistenceEngine.addApplication(app) sender ! RegisteredApplication(app.id, masterUrl) schedule() @@ -262,15 +258,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act if (canCompleteRecovery) { completeRecovery() } } - case Terminated(actor) => { - // The disconnected actor could've been either a worker or an app; remove whichever of - // those we have an entry for in the corresponding actor hashmap - logInfo(s"$actor got terminated, removing it.") - actorToWorker.get(actor).foreach(removeWorker) - actorToApp.get(actor).foreach(finishApplication) - if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() } - } - case DisassociatedEvent(_, address, _) => { // The disconnected client could've been either a worker or an app; remove whichever it was logInfo(s"$address got disassociated, removing it.") @@ -438,8 +425,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act exec.id, ExecutorState.LOST, Some("worker lost"), None) exec.application.removeExecutor(exec) } - context.stop(worker.actor) - context.unwatch(worker.actor) persistenceEngine.removeWorker(worker) } @@ -502,8 +487,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act app.driver ! ApplicationRemoved(state.toString) } persistenceEngine.removeApplication(app) - context.stop(app.driver) - context.unwatch(app.driver) schedule() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 3a7d0b859b..0a183afd8e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -73,7 +73,6 @@ private[spark] class Worker( val masterLock: Object = new Object() var master: ActorSelection = null - var prevMaster: ActorRef = null var activeMasterUrl: String = "" var activeMasterWebUiUrl : String = "" @volatile var registered = false @@ -173,8 +172,6 @@ private[spark] class Worker( case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) registered = true - context.watch(sender) // remote death watch for master - prevMaster = sender changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) @@ -185,8 +182,6 @@ private[spark] class Worker( case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) - context.unwatch(prevMaster) - prevMaster = sender changeMaster(masterUrl, masterWebUiUrl) val execs = executors.values. @@ -245,10 +240,6 @@ private[spark] class Worker( } } - case Terminated(actor_) => - logInfo(s"$actor_ terminated !") - masterDisconnected() - case x: DisassociatedEvent => logInfo(s"$x Disassociated !") masterDisconnected() diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 2818a775d0..dcb12bed4e 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -51,7 +51,6 @@ private[spark] class CoarseGrainedExecutorBackend( override def receive = { case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") - context.watch(sender) //Start watching for terminated messages. // Make this host instead of hostPort ? executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) @@ -76,10 +75,6 @@ private[spark] class CoarseGrainedExecutorBackend( executor.killTask(taskId) } - case Terminated(actor) => - logError(s"Driver $actor terminated, Shutting down.") - System.exit(1) - case x: DisassociatedEvent => logError(s"Driver $x disassociated! Shutting down.") System.exit(1) 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 e316f6b41f..d614dcbdd8 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 @@ -73,7 +73,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac } else { logInfo("Registered executor: " + sender + " with ID " + executorId) sender ! RegisteredExecutor(sparkProperties) - context.watch(sender) executorActor(executorId) = sender executorHost(executorId) = Utils.parseHostPort(hostPort)._1 freeCores(executorId) = cores @@ -118,9 +117,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac removeExecutor(executorId, reason) sender ! true - case Terminated(actor) => - actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated")) - case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) 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 90a5387b2b..23e9b735f3 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -46,20 +46,15 @@ private[spark] object AkkaUtils { val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "60").toInt val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble - // Since we have our own Heart Beat mechanism and TCP already tracks connections. - // Using this makes very little sense. So setting this to a relatively larger value suffices. - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "3").toInt + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "5").toInt val akkaConf = ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] |akka.stdout-loglevel = "ERROR" - |akka.remote.watch-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s - |akka.remote.watch-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.watch-failure-detector.threshold = $akkaFailureDetector - |akka.remote.transport-failure-detector.heartbeat-interval = 30 s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = ${akkaHeartBeatPauses + 10} s + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" diff --git a/spark-class b/spark-class index 78d6e073b1..713404d077 100755 --- a/spark-class +++ b/spark-class @@ -136,3 +136,4 @@ fi exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" + -- cgit v1.2.3 From d092a8cc6a75ed4d38cf1c3c065dfe05ca4f695d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 26 Nov 2013 14:26:23 +0530 Subject: Fixed compile time warnings and formatting post merge. --- .../apache/spark/deploy/master/RecoveryState.scala | 4 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 4 +- .../org/apache/spark/util/MetadataCleaner.scala | 3 +- .../org/apache/spark/storage/BlockIdSuite.scala | 2 +- .../scala/org/apache/spark/repl/ReplSuite.scala | 139 +++++++++++---------- 5 files changed, 79 insertions(+), 73 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala index b91be821f0..256a5a7c28 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala @@ -17,9 +17,7 @@ package org.apache.spark.deploy.master -private[spark] object RecoveryState - extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") { - +private[spark] object RecoveryState extends Enumeration { type MasterState = Value val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 5f6407aadc..da18d45e65 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -553,7 +553,7 @@ abstract class RDD[T: ClassTag]( (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false) - def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest] + def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean) (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning) @@ -563,7 +563,7 @@ abstract class RDD[T: ClassTag]( (f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false) - def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest] + def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag] (rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean) (f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] = new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning) diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 67a7f87a5c..7b41ef89f1 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -55,8 +55,7 @@ class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, clea } } -object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext", "HttpBroadcast", "DagScheduler", "ResultTask", - "ShuffleMapTask", "BlockManager", "DiskBlockManager", "BroadcastVars") { +object MetadataCleanerType extends Enumeration { val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala index cb76275e39..b647e8a672 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala @@ -39,7 +39,7 @@ class BlockIdSuite extends FunSuite { fail() } catch { case e: IllegalStateException => // OK - case _ => fail() + case _: Throwable => fail() } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index c230a03298..daaa2a0305 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -79,23 +79,25 @@ class ReplSuite extends FunSuite { } test("simple foreach with accumulator") { - val output = runInterpreter("local", """ - |val accum = sc.accumulator(0) - |sc.parallelize(1 to 10).foreach(x => accum += x) - |accum.value - """.stripMargin) + val output = runInterpreter("local", + """ + |val accum = sc.accumulator(0) + |sc.parallelize(1 to 10).foreach(x => accum += x) + |accum.value + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res1: Int = 55", output) } test("external vars") { - val output = runInterpreter("local", """ - |var v = 7 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) - |v = 10 - |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |var v = 7 + |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + |v = 10 + |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) @@ -103,35 +105,38 @@ class ReplSuite extends FunSuite { } test("external classes") { - val output = runInterpreter("local", """ - |class C { - |def foo = 5 - |} - |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |class C { + |def foo = 5 + |} + |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 50", output) } test("external functions") { - val output = runInterpreter("local", """ - |def double(x: Int) = x + x - |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |def double(x: Int) = x + x + |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 110", output) } test("external functions that access vars") { - val output = runInterpreter("local", """ - |var v = 7 - |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) - |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) - """.stripMargin) + val output = runInterpreter("local", + """ + |var v = 7 + |def getV() = v + |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |v = 10 + |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) @@ -142,13 +147,14 @@ class ReplSuite extends FunSuite { // Test that the value that a broadcast var had when it was created is used, // even if that variable is then modified in the driver program // TODO: This doesn't actually work for arrays when we run in local mode! - val output = runInterpreter("local", """ - |var array = new Array[Int](5) - |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect - |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect - """.stripMargin) + val output = runInterpreter("local", + """ + |var array = new Array[Int](5) + |val broadcastArray = sc.broadcast(array) + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |array(0) = 5 + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Array[Int] = Array(0, 0, 0, 0, 0)", output) @@ -162,12 +168,13 @@ class ReplSuite extends FunSuite { out.write("What's up?\n") out.write("Goodbye\n") out.close() - val output = runInterpreter("local", """ - |var file = sc.textFile("%s/input").cache() - |file.count() - |file.count() - |file.count() - """.stripMargin.format(tempDir.getAbsolutePath)) + val output = runInterpreter("local", + """ + |var file = sc.textFile("%s/input").cache() + |file.count() + |file.count() + |file.count() + """.stripMargin.format(tempDir.getAbsolutePath)) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Long = 3", output) @@ -176,18 +183,19 @@ class ReplSuite extends FunSuite { } test("local-cluster mode") { - val output = runInterpreter("local-cluster[1,1,512]", """ - |var v = 7 - |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) - |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) - |var array = new Array[Int](5) - |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect - |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect - """.stripMargin) + val output = runInterpreter("local-cluster[1,1,512]", + """ + |var v = 7 + |def getV() = v + |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |v = 10 + |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |var array = new Array[Int](5) + |val broadcastArray = sc.broadcast(array) + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |array(0) = 5 + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) @@ -198,18 +206,19 @@ class ReplSuite extends FunSuite { if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test("running on Mesos") { - val output = runInterpreter("localquiet", """ - |var v = 7 - |def getV() = v - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) - |v = 10 - |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) - |var array = new Array[Int](5) - |val broadcastArray = sc.broadcast(array) - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect - |array(0) = 5 - |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect - """.stripMargin) + val output = runInterpreter("localquiet", + """ + |var v = 7 + |def getV() = v + |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |v = 10 + |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_) + |var array = new Array[Int](5) + |val broadcastArray = sc.broadcast(array) + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + |array(0) = 5 + |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect + """.stripMargin) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Int = 70", output) -- cgit v1.2.3 From 560e44a8e1d5a2cf42bf640090016f6201c6fbd7 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 26 Nov 2013 17:50:29 +0530 Subject: Restored master address for client. --- .../main/scala/org/apache/spark/deploy/client/Client.scala | 11 +++++++---- .../main/scala/org/apache/spark/deploy/master/Master.scala | 4 ++-- 2 files changed, 9 insertions(+), 6 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 408692ec9c..f60e56d959 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 @@ -27,7 +27,7 @@ import akka.pattern.AskTimeoutException import akka.pattern.ask import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent, AssociationErrorEvent} -import org.apache.spark.Logging +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 @@ -49,7 +49,7 @@ private[spark] class Client( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 - var prevMaster: ActorRef = null // set for unwatching, when it fails. + var masterAddress: Address = null var actor: ActorRef = null var appId: String = null var registered = false @@ -103,11 +103,14 @@ private[spark] class Client( def changeMaster(url: String) { activeMasterUrl = url master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) + masterAddress = activeMasterUrl match { + case Master.sparkUrlRegex(host, port) => Address("akka.tcp", Master.systemName, host, port.toInt) + case x => throw new SparkException("Invalid spark URL:"+x) + } } override def receive = { case RegisteredApplication(appId_, masterUrl) => - prevMaster = sender appId = appId_ registered = true changeMaster(masterUrl) @@ -137,7 +140,7 @@ private[spark] class Client( alreadyDisconnected = false sender ! MasterChangeAcknowledged(appId) - case DisassociatedEvent(_, address, _) => + case DisassociatedEvent(_, address, _) if address == masterAddress => logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() 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 81fb5c4e43..0e2b461b13 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 @@ -517,9 +517,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } private[spark] object Master { - private val systemName = "sparkMaster" + val systemName = "sparkMaster" private val actorName = "Master" - private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r + val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { val args = new MasterArguments(argStrings) -- cgit v1.2.3 From 54862af5ee813030ead80ec097f48620ddb974fc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 27 Nov 2013 14:26:28 +0530 Subject: Improvements from the review comments and followed Boy Scout Rule. --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 8 ++------ .../scala/org/apache/spark/deploy/master/Master.scala | 8 +++----- .../scala/org/apache/spark/deploy/worker/Worker.scala | 9 +++++++-- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/cluster/ClusterScheduler.scala | 4 +--- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 5 +---- .../org/apache/spark/storage/BlockManagerMaster.scala | 15 +++------------ core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 5 +++-- docs/configuration.md | 4 ++-- 9 files changed, 23 insertions(+), 37 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a686b534da..88a7f24884 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -21,15 +21,11 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashSet +import scala.concurrent.Await +import scala.concurrent.duration._ import akka.actor._ -import scala.concurrent.Await import akka.pattern.ask -import akka.remote._ - -import scala.concurrent.duration.Duration -import akka.util.Timeout -import scala.concurrent.duration._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId 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 0e2b461b13..c627dd3806 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 @@ -39,9 +39,6 @@ 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.deploy.DeployMessages.RegisterWorkerFailed -import org.apache.spark.deploy.DeployMessages.KillExecutor -import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { import context.dispatcher @@ -159,7 +156,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act System.exit(0) } - case RegisterWorker(id, host, workerPort, cores, memory, webUiPort, publicAddress) => { + case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) => { logInfo("Registering worker %s:%d with %d cores, %s RAM".format( host, workerPort, cores, Utils.megabytesToString(memory))) if (state == RecoveryState.STANDBY) { @@ -167,7 +164,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act } else if (idToWorker.contains(id)) { sender ! RegisterWorkerFailed("Duplicate worker ID") } else { - val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress) + val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, + sender, workerWebUiPort, publicAddress) registerWorker(worker) persistenceEngine.addWorker(worker) sender ! RegisteredWorker(masterUrl, masterWebUiUrl) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 0a183afd8e..808b54c0af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -27,7 +27,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.remote.{ DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -73,6 +73,7 @@ private[spark] class Worker( val masterLock: Object = new Object() var master: ActorSelection = null + var masterAddress: Address = null var activeMasterUrl: String = "" var activeMasterWebUiUrl : String = "" @volatile var registered = false @@ -136,6 +137,10 @@ private[spark] class Worker( activeMasterUrl = url activeMasterWebUiUrl = uiUrl master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) + masterAddress = activeMasterUrl match { + case Master.sparkUrlRegex(_host, _port) => Address("akka.tcp", Master.systemName, _host, _port.toInt) + case x => throw new SparkException("Invalid spark URL:"+x) + } connected = true } } @@ -240,7 +245,7 @@ private[spark] class Worker( } } - case x: DisassociatedEvent => + case x: DisassociatedEvent if x.remoteAddress == masterAddress => logInfo(s"$x Disassociated !") masterDisconnected() diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index fdea3f6f88..773e9ec182 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -22,7 +22,6 @@ import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.reflect.ClassTag @@ -115,6 +114,7 @@ class DAGScheduler( private val eventProcessActor: ActorRef = env.actorSystem.actorOf(Props(new Actor { override def preStart() { + import context.dispatcher context.system.scheduler.schedule(RESUBMIT_TIMEOUT, RESUBMIT_TIMEOUT) { if (failed.size > 0) { resubmitFailedStages() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala index 7c9d6a93e4..8056cb2597 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala @@ -24,8 +24,6 @@ import java.util.{TimerTask, Timer} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet - -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import org.apache.spark._ @@ -123,7 +121,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext) if (System.getProperty("spark.speculation", "false").toBoolean) { logInfo("Starting speculative execution thread") - + import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, SPECULATION_INTERVAL milliseconds) { checkSpeculatableTasks() 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 d614dcbdd8..f5e8766f6d 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 @@ -25,7 +25,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.pattern.ask -import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{SparkException, Logging, TaskState} import org.apache.spark.scheduler.TaskDescription @@ -52,7 +52,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac private val executorAddress = new HashMap[String, Address] private val executorHost = new HashMap[String, String] private val freeCores = new HashMap[String, Int] - private val actorToExecutorId = new HashMap[ActorRef, String] private val addressToExecutorId = new HashMap[Address, String] override def preStart() { @@ -77,7 +76,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac executorHost(executorId) = Utils.parseHostPort(hostPort)._1 freeCores(executorId) = cores executorAddress(executorId) = sender.path.address - actorToExecutorId(sender) = executorId addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) makeOffers() @@ -147,7 +145,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac if (executorActor.contains(executorId)) { logInfo("Executor " + executorId + " disconnected, so removing it") val numCores = freeCores(executorId) - actorToExecutorId -= executorActor(executorId) addressToExecutorId -= executorAddress(executorId) executorActor -= executorId executorHost -= executorId 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 a4aa316e4b..e5de16fc01 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -17,24 +17,15 @@ package org.apache.spark.storage -import akka.actor._ -import scala.concurrent.Await -import scala.concurrent.Future +import scala.concurrent.{Await, Future} +import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global +import akka.actor._ import akka.pattern.ask -import scala.concurrent.duration._ import org.apache.spark.{Logging, SparkException} import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.storage.BlockManagerMessages.GetLocations -import org.apache.spark.storage.BlockManagerMessages.GetLocationsMultipleBlockIds -import org.apache.spark.storage.BlockManagerMessages.RegisterBlockManager -import org.apache.spark.storage.BlockManagerMessages.HeartBeat -import org.apache.spark.storage.BlockManagerMessages.RemoveExecutor -import org.apache.spark.storage.BlockManagerMessages.GetPeers -import org.apache.spark.storage.BlockManagerMessages.RemoveBlock -import org.apache.spark.storage.BlockManagerMessages.RemoveRdd private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, ActorSelection]) extends Logging { 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 23e9b735f3..3444d8fdfe 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -44,8 +44,9 @@ private[spark] object AkkaUtils { val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.pauses", "60").toInt - val akkaFailureDetector = System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble + val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "60").toInt + val akkaFailureDetector = + System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "5").toInt val akkaConf = ConfigFactory.parseString( diff --git a/docs/configuration.md b/docs/configuration.md index 25e7cecbfc..4d1a987f64 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -281,7 +281,7 @@ Apart from these, the following properties are also available, and may be useful - + -- cgit v1.2.3 From 3ec5d7476690a5f8a011ec3eaa2d9003f9559b12 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 28 Nov 2013 13:02:28 +0530 Subject: Fixed the broken build. --- core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 02d75eccc5..688c310ee9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -90,12 +90,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { // Compute the minimum and the maxium val (max: Double, min: Double) = self.mapPartitions { items => - Iterator(items.foldRight(-1/0.0, Double.NaN)((e: Double, x: Pair[Double, Double]) => + Iterator(items.foldRight(Double.NegativeInfinity, + Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) => (x._1.max(e), x._2.min(e)))) }.reduce { (maxmin1, maxmin2) => (maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2)) } - if (max.isNaN() || max.isInfinity || min.isInfinity ) { + if (min.isNaN || max.isNaN || max.isInfinity || min.isInfinity ) { throw new UnsupportedOperationException( "Histogram on either an empty RDD or RDD containing +/-infinity or NaN") } -- cgit v1.2.3 From 1bc83ca79187979f58385d3f28236111217174e0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 28 Nov 2013 16:07:15 +0530 Subject: Changed defaults for akka to almost disable failure detector. --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 3444d8fdfe..5df8213d74 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -39,21 +39,22 @@ private[spark] object AkkaUtils { val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt - val akkaTimeout = System.getProperty("spark.akka.timeout", "60").toInt + val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" - val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "60").toInt + val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = - System.getProperty("spark.akka.failure-detector.threshold", "12.0").toDouble - val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "5").toInt + System.getProperty("spark.akka.failure-detector.threshold", "300.0").toDouble + val akkaHeartBeatInterval = System.getProperty("spark.akka.heartbeat.interval", "1000").toInt val akkaConf = ConfigFactory.parseString( s""" |akka.daemonic = on |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] |akka.stdout-loglevel = "ERROR" + |akka.jvm-exit-on-fatal-error = off |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector @@ -61,6 +62,7 @@ private[spark] object AkkaUtils { |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port + |akka.remote.netty.tcp.tcp-nodelay = on |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB |akka.remote.netty.tcp.execution-pool-size = $akkaThreads -- cgit v1.2.3 From 5b11028a0479623f41e95a41825a9bdfc944b323 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 2 Dec 2013 10:41:26 +0530 Subject: Made akka capable of tolerating fatal exceptions and moving on. --- .../scala/org/apache/spark/util/AkkaUtils.scala | 4 +- .../org/apache/spark/util/SparkActorSystem.scala | 112 +++++++++++++++++++++ 2 files changed, 114 insertions(+), 2 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala (limited to 'core/src/main/scala/org/apache') 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 5df8213d74..407e9ffe90 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import akka.actor.{ActorSystem, ExtendedActorSystem} +import akka.actor.{SparkActorSystem, ActorSystem, ExtendedActorSystem} import com.typesafe.config.ConfigFactory import scala.concurrent.duration._ import scala.concurrent.Await @@ -70,7 +70,7 @@ private[spark] object AkkaUtils { |akka.remote.log-remote-lifecycle-events = $lifecycleEvents """.stripMargin) - val actorSystem = ActorSystem(name, akkaConf) + val actorSystem = SparkActorSystem(name, akkaConf) val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get diff --git a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala new file mode 100644 index 0000000000..461e7ab08f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala @@ -0,0 +1,112 @@ +/** + * Copyright (C) 2009-2013 Typesafe Inc. + */ + +package akka.actor + +import com.typesafe.config.Config +import akka.util._ +import scala.util.control.{NonFatal, ControlThrowable} + +/** + * An actorSystem specific to spark. It has an additional feature of letting spark tolerate + * fatal exceptions. + */ +object SparkActorSystem { + + def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader()) + + def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = + new SparkActorSystemImpl(name, config, classLoader).start() + + /** + * INTERNAL API + */ + private[akka] def findClassLoader(): ClassLoader = { + def findCaller(get: Int ⇒ Class[_]): ClassLoader = + Iterator.from(2 /*is the magic number, promise*/).map(get) dropWhile { + c ⇒ + c != null && + (c.getName.startsWith("akka.actor.ActorSystem") || + c.getName.startsWith("scala.Option") || + c.getName.startsWith("scala.collection.Iterator") || + c.getName.startsWith("akka.util.Reflect")) + } next() match { + case null ⇒ getClass.getClassLoader + case c ⇒ c.getClassLoader + } + + Option(Thread.currentThread.getContextClassLoader) orElse + (Reflect.getCallerClass map findCaller) getOrElse + getClass.getClassLoader + } +} + +private[akka] class SparkActorSystemImpl(override val name: String, + applicationConfig: Config, + classLoader: ClassLoader) + extends ActorSystemImpl(name, applicationConfig, classLoader) { + + protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = + new Thread.UncaughtExceptionHandler() { + def uncaughtException(thread: Thread, cause: Throwable): Unit = { + cause match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable + ⇒ log.error(cause, "Uncaught error from thread [{}]", thread.getName) + case _ ⇒ + if (settings.JvmExitOnFatalError) { + try { + log.error(cause, "Uncaught error from thread [{}] shutting down JVM since " + + "'akka.jvm-exit-on-fatal-error' is enabled", thread.getName) + import System.err + err.print("Uncaught error from thread [") + err.print(thread.getName) + err.print("] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled for " + + "ActorSystem[") + err.print(name) + err.println("]") + cause.printStackTrace(System.err) + System.err.flush() + } finally { + System.exit(-1) + } + } else { + log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + + "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) + //shutdown() //TODO make it configurable + if (thread.isAlive) log.error("Thread is still alive") + else { + log.error("Thread is dead") + } + } + } + } + } + + override def stop(actor: ActorRef): Unit = { + val path = actor.path + val guard = guardian.path + val sys = systemGuardian.path + path.parent match { + case `guard` ⇒ guardian ! StopChild(actor) + case `sys` ⇒ systemGuardian ! StopChild(actor) + case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() + } + } + + + override def /(actorName: String): ActorPath = guardian.path / actorName + + override def /(path: Iterable[String]): ActorPath = guardian.path / path + + private lazy val _start: this.type = { + // the provider is expected to start default loggers, LocalActorRefProvider does this + provider.init(this) + this + } + + override def start(): this.type = _start + + override def toString: String = lookupRoot.path.root.address.toString + +} -- cgit v1.2.3 From f6c8c1c7b686a010ffcec238db14eda34f1645f1 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 2 Dec 2013 11:42:53 -0800 Subject: Cleanup and documentation of SparkActorSystem --- .../org/apache/spark/util/SparkActorSystem.scala | 114 ++++++--------------- 1 file changed, 29 insertions(+), 85 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala index 461e7ab08f..a679fd6142 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala @@ -2,111 +2,55 @@ * Copyright (C) 2009-2013 Typesafe Inc. */ +// Must be in akka.actor package as ActorSystemImpl is protected[akka]. package akka.actor +import scala.util.control.{ControlThrowable, NonFatal} + import com.typesafe.config.Config -import akka.util._ -import scala.util.control.{NonFatal, ControlThrowable} /** - * An actorSystem specific to spark. It has an additional feature of letting spark tolerate - * fatal exceptions. + * An ActorSystem specific to Spark. Based off of [[akka.actor.ActorSystem]]. + * The only change from the default system is that we do not shut down the ActorSystem + * in the event of a fatal exception. This is necessary as Spark is allowed to recover + * from fatal exceptions (see [[org.apache.spark.executor.Executor]]). */ object SparkActorSystem { - - def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader()) + def apply(name: String, config: Config): ActorSystem = + apply(name, config, ActorSystem.findClassLoader()) def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = new SparkActorSystemImpl(name, config, classLoader).start() - - /** - * INTERNAL API - */ - private[akka] def findClassLoader(): ClassLoader = { - def findCaller(get: Int ⇒ Class[_]): ClassLoader = - Iterator.from(2 /*is the magic number, promise*/).map(get) dropWhile { - c ⇒ - c != null && - (c.getName.startsWith("akka.actor.ActorSystem") || - c.getName.startsWith("scala.Option") || - c.getName.startsWith("scala.collection.Iterator") || - c.getName.startsWith("akka.util.Reflect")) - } next() match { - case null ⇒ getClass.getClassLoader - case c ⇒ c.getClassLoader - } - - Option(Thread.currentThread.getContextClassLoader) orElse - (Reflect.getCallerClass map findCaller) getOrElse - getClass.getClassLoader - } } -private[akka] class SparkActorSystemImpl(override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) +private[akka] class SparkActorSystemImpl( + override val name: String, + applicationConfig: Config, + classLoader: ClassLoader) extends ActorSystemImpl(name, applicationConfig, classLoader) { - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = + protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { + val fallbackHandler = super.uncaughtExceptionHandler + new Thread.UncaughtExceptionHandler() { def uncaughtException(thread: Thread, cause: Throwable): Unit = { - cause match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable - ⇒ log.error(cause, "Uncaught error from thread [{}]", thread.getName) - case _ ⇒ - if (settings.JvmExitOnFatalError) { - try { - log.error(cause, "Uncaught error from thread [{}] shutting down JVM since " + - "'akka.jvm-exit-on-fatal-error' is enabled", thread.getName) - import System.err - err.print("Uncaught error from thread [") - err.print(thread.getName) - err.print("] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled for " + - "ActorSystem[") - err.print(name) - err.println("]") - cause.printStackTrace(System.err) - System.err.flush() - } finally { - System.exit(-1) - } - } else { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) - //shutdown() //TODO make it configurable - if (thread.isAlive) log.error("Thread is still alive") - else { - log.error("Thread is dead") - } - } + if (isFatalError(cause) && !settings.JvmExitOnFatalError) { + log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + + "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) + //shutdown() //TODO make it configurable + } else { + fallbackHandler.uncaughtException(thread, cause) } } } - - override def stop(actor: ActorRef): Unit = { - val path = actor.path - val guard = guardian.path - val sys = systemGuardian.path - path.parent match { - case `guard` ⇒ guardian ! StopChild(actor) - case `sys` ⇒ systemGuardian ! StopChild(actor) - case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() - } } - - override def /(actorName: String): ActorPath = guardian.path / actorName - - override def /(path: Iterable[String]): ActorPath = guardian.path / path - - private lazy val _start: this.type = { - // the provider is expected to start default loggers, LocalActorRefProvider does this - provider.init(this) - this + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } } - - override def start(): this.type = _start - - override def toString: String = lookupRoot.path.root.address.toString - } -- cgit v1.2.3 From 0f24576c08a361f323b7ad9babfd5d8431d57df0 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 2 Dec 2013 11:42:53 -0800 Subject: Cleanup and documentation of SparkActorSystem --- .../org/apache/spark/util/SparkActorSystem.scala | 114 ++++++--------------- 1 file changed, 29 insertions(+), 85 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala index 461e7ab08f..d329063e43 100644 --- a/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkActorSystem.scala @@ -2,111 +2,55 @@ * Copyright (C) 2009-2013 Typesafe Inc. */ +// Must be in akka.actor package as ActorSystemImpl is protected[akka]. package akka.actor +import scala.util.control.{ControlThrowable, NonFatal} + import com.typesafe.config.Config -import akka.util._ -import scala.util.control.{NonFatal, ControlThrowable} /** - * An actorSystem specific to spark. It has an additional feature of letting spark tolerate - * fatal exceptions. + * An ActorSystem specific to Spark. Based off of [[akka.actor.ActorSystem]]. + * The only change from the default system is that we do not shut down the ActorSystem + * in the event of a fatal exception. This is necessary as Spark is allowed to recover + * from fatal exceptions (see [[org.apache.spark.executor.Executor]]). */ object SparkActorSystem { - - def apply(name: String, config: Config): ActorSystem = apply(name, config, findClassLoader()) + def apply(name: String, config: Config): ActorSystem = + apply(name, config, ActorSystem.findClassLoader()) def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = new SparkActorSystemImpl(name, config, classLoader).start() - - /** - * INTERNAL API - */ - private[akka] def findClassLoader(): ClassLoader = { - def findCaller(get: Int ⇒ Class[_]): ClassLoader = - Iterator.from(2 /*is the magic number, promise*/).map(get) dropWhile { - c ⇒ - c != null && - (c.getName.startsWith("akka.actor.ActorSystem") || - c.getName.startsWith("scala.Option") || - c.getName.startsWith("scala.collection.Iterator") || - c.getName.startsWith("akka.util.Reflect")) - } next() match { - case null ⇒ getClass.getClassLoader - case c ⇒ c.getClassLoader - } - - Option(Thread.currentThread.getContextClassLoader) orElse - (Reflect.getCallerClass map findCaller) getOrElse - getClass.getClassLoader - } } -private[akka] class SparkActorSystemImpl(override val name: String, - applicationConfig: Config, - classLoader: ClassLoader) +private[akka] class SparkActorSystemImpl( + override val name: String, + applicationConfig: Config, + classLoader: ClassLoader) extends ActorSystemImpl(name, applicationConfig, classLoader) { - protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = + protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = { + val fallbackHandler = super.uncaughtExceptionHandler + new Thread.UncaughtExceptionHandler() { def uncaughtException(thread: Thread, cause: Throwable): Unit = { - cause match { - case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable - ⇒ log.error(cause, "Uncaught error from thread [{}]", thread.getName) - case _ ⇒ - if (settings.JvmExitOnFatalError) { - try { - log.error(cause, "Uncaught error from thread [{}] shutting down JVM since " + - "'akka.jvm-exit-on-fatal-error' is enabled", thread.getName) - import System.err - err.print("Uncaught error from thread [") - err.print(thread.getName) - err.print("] shutting down JVM since 'akka.jvm-exit-on-fatal-error' is enabled for " + - "ActorSystem[") - err.print(name) - err.println("]") - cause.printStackTrace(System.err) - System.err.flush() - } finally { - System.exit(-1) - } - } else { - log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + - "ActorSystem tolerating and continuing.... [{}]", thread.getName, name) - //shutdown() //TODO make it configurable - if (thread.isAlive) log.error("Thread is still alive") - else { - log.error("Thread is dead") - } - } + if (isFatalError(cause) && !settings.JvmExitOnFatalError) { + log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + + "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) + //shutdown() //TODO make it configurable + } else { + fallbackHandler.uncaughtException(thread, cause) } } } - - override def stop(actor: ActorRef): Unit = { - val path = actor.path - val guard = guardian.path - val sys = systemGuardian.path - path.parent match { - case `guard` ⇒ guardian ! StopChild(actor) - case `sys` ⇒ systemGuardian ! StopChild(actor) - case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() - } } - - override def /(actorName: String): ActorPath = guardian.path / actorName - - override def /(path: Iterable[String]): ActorPath = guardian.path / path - - private lazy val _start: this.type = { - // the provider is expected to start default loggers, LocalActorRefProvider does this - provider.init(this) - this + def isFatalError(e: Throwable): Boolean = { + e match { + case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable => + false + case _ => + true + } } - - override def start(): this.type = _start - - override def toString: String = lookupRoot.path.root.address.toString - } -- cgit v1.2.3 From 09e8be9a6225203337a01e618851e807a1482603 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 3 Dec 2013 11:27:45 +0530 Subject: Made running SparkActorSystem specific to executors only. --- .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 3 ++- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 11 +++++++++-- 2 files changed, 11 insertions(+), 3 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index dcb12bed4e..406e015f08 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -97,7 +97,8 @@ private[spark] object CoarseGrainedExecutorBackend { // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, + useSparkAS = true) // set it val sparkHostPort = hostname + ":" + boundPort System.setProperty("spark.hostPort", sparkHostPort) 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 407e9ffe90..f3e2644a58 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -35,7 +35,9 @@ private[spark] object AkkaUtils { * Note: the `name` parameter is important, as even if a client sends a message to right * host + port, if the system name is incorrect, Akka will drop the message. */ - def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = { + def createActorSystem(name: String, host: String, port: Int, + useSparkAS: Boolean = false): (ActorSystem, Int) = { + val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt @@ -70,7 +72,12 @@ private[spark] object AkkaUtils { |akka.remote.log-remote-lifecycle-events = $lifecycleEvents """.stripMargin) - val actorSystem = SparkActorSystem(name, akkaConf) + val actorSystem = if (useSparkAS) { + SparkActorSystem(name, akkaConf) + } + else { + ActorSystem(name, akkaConf) + } val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider val boundPort = provider.getDefaultAddress.port.get -- cgit v1.2.3 From 4e70480038e9654426876e8e6b2fc356b7f0c8ca Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 6 Dec 2013 12:25:32 +0530 Subject: A left over akka -> akka.tcp changes --- .../scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala | 2 +- yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index e000531a26..e8fecec4a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -36,7 +36,7 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 421a83c87a..b67e068844 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -168,7 +168,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration) exte System.setProperty("spark.driver.host", driverHost) System.setProperty("spark.driver.port", driverPort.toString) - val driverUrl = "akka://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") -- cgit v1.2.3 From 94b5881ee9d6c67f096ea9c2891a63978f256394 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 6 Dec 2013 00:22:00 -0800 Subject: Fix long lines --- core/src/main/scala/org/apache/spark/deploy/client/Client.scala | 6 ++++-- core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 6 ++++-- .../main/scala/org/apache/spark/storage/BlockObjectWriter.scala | 2 +- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 8 +++----- 4 files changed, 12 insertions(+), 10 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 f60e56d959..d0d65ca72a 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 @@ -104,8 +104,10 @@ private[spark] class Client( activeMasterUrl = url master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) masterAddress = activeMasterUrl match { - case Master.sparkUrlRegex(host, port) => Address("akka.tcp", Master.systemName, host, port.toInt) - case x => throw new SparkException("Invalid spark URL:"+x) + case Master.sparkUrlRegex(host, port) => + Address("akka.tcp", Master.systemName, host, port.toInt) + case x => + throw new SparkException("Invalid spark URL: " + x) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 808b54c0af..87531b6719 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -138,8 +138,10 @@ private[spark] class Worker( activeMasterWebUiUrl = uiUrl master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) masterAddress = activeMasterUrl match { - case Master.sparkUrlRegex(_host, _port) => Address("akka.tcp", Master.systemName, _host, _port.toInt) - case x => throw new SparkException("Invalid spark URL:"+x) + case Master.sparkUrlRegex(_host, _port) => + Address("akka.tcp", Master.systemName, _host, _port.toInt) + case x => + throw new SparkException("Invalid spark URL: " + x) } connected = true } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index df33f6bfb0..b4451fc7b8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -44,7 +44,7 @@ abstract class BlockObjectWriter(val blockId: BlockId) { * Flush the partial writes and commit them as a single atomic block. Return the * number of bytes written for this commit. */ - def commit(): LongSpark + def commit(): Long /** * Reverts writes that haven't been flushed yet. Callers should invoke this function 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 9f3f163277..74133cef6c 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -17,11 +17,8 @@ package org.apache.spark.util -import akka.actor.{IndestructibleActorSystem, ActorSystem, ExtendedActorSystem} +import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory -import scala.concurrent.duration._ -import scala.concurrent.Await -import akka.remote.RemoteActorRefProvider /** * Various utility classes for working with Akka. @@ -47,7 +44,8 @@ private[spark] object AkkaUtils { val akkaTimeout = System.getProperty("spark.akka.timeout", "100").toInt val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt - val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" + val lifecycleEvents = + if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off" val akkaHeartBeatPauses = System.getProperty("spark.akka.heartbeat.pauses", "600").toInt val akkaFailureDetector = -- cgit v1.2.3 From 14a1df65729a3c7e5fd54f79424626a540aef5d4 Mon Sep 17 00:00:00 2001 From: Grega Kespret Date: Mon, 9 Dec 2013 10:39:02 +0100 Subject: Fix for spark.task.maxFailures not enforced correctly. --- .../org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 { -- cgit v1.2.3 From 5b74609d97cce9c350e959547e7fc524182a457c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 9 Dec 2013 16:40:30 -0800 Subject: License headers --- .../org/apache/spark/storage/StoragePerfTester.scala | 17 +++++++++++++++++ .../apache/spark/deploy/worker/ExecutorRunnerTest.scala | 17 +++++++++++++++++ .../apache/spark/storage/DiskBlockManagerSuite.scala | 17 +++++++++++++++++ .../apache/spark/util/collection/OpenHashMapSuite.scala | 17 +++++++++++++++++ .../apache/spark/util/collection/OpenHashSetSuite.scala | 17 +++++++++++++++++ .../util/collection/PrimitiveKeyOpenHashMapSuite.scala | 17 +++++++++++++++++ python/test_support/userlibrary.py | 17 +++++++++++++++++ 7 files changed, 119 insertions(+) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala index 1e4db4f66b..d52b3d8284 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.util.concurrent.atomic.AtomicLong diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 8f0954122b..4cb4ddc9cd 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -1,3 +1,20 @@ +/* + * 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.deploy.worker import java.io.File diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index ef4c4c0f14..070982e798 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.storage import java.io.{FileWriter, File} diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 63e874fed3..e9b62ea70d 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.util.collection import scala.collection.mutable.HashSet diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index 4768a1e60b..1b24f8f287 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.util.collection import org.scalatest.FunSuite diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala index 2220b4f0d5..3b60decee9 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala @@ -1,3 +1,20 @@ +/* + * 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.util.collection import scala.collection.mutable.HashSet diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py index 5bb6f5009f..8e4a6292bc 100755 --- a/python/test_support/userlibrary.py +++ b/python/test_support/userlibrary.py @@ -1,3 +1,20 @@ +# +# 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. +# + """ Used to test shipping of code depenencies with SparkContext.addPyFile(). """ -- cgit v1.2.3 From 17db6a9041d5e83d7b6fe47f9c36758d0613fcd6 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 10 Dec 2013 11:34:10 +0530 Subject: Style fixes and addressed review comments at #221 --- assembly/pom.xml | 14 +++++++------- bagel/pom.xml | 10 +++++----- core/pom.xml | 18 +++++++++--------- .../scala/org/apache/spark/MapOutputTracker.scala | 9 ++++----- .../main/scala/org/apache/spark/Partitioner.scala | 6 +++--- .../apache/spark/deploy/worker/ui/WorkerWebUI.scala | 14 +++++--------- .../scala/org/apache/spark/rdd/CheckpointRDD.scala | 11 ++++------- .../main/scala/org/apache/spark/rdd/JdbcRDD.scala | 1 + .../main/scala/org/apache/spark/rdd/MappedRDD.scala | 3 ++- .../org/apache/spark/rdd/OrderedRDDFunctions.scala | 3 ++- .../scala/org/apache/spark/rdd/ShuffledRDD.scala | 3 ++- .../apache/spark/storage/BlockManagerMaster.scala | 7 +++---- .../spark/util/IndestructibleActorSystem.scala | 17 +++++++++++++++-- examples/pom.xml | 20 ++++++++++---------- mllib/pom.xml | 10 +++++----- pom.xml | 17 ++++++++--------- repl-bin/pom.xml | 6 +++--- repl/pom.xml | 14 +++++++------- streaming/pom.xml | 16 ++++++++-------- .../spark/streaming/receivers/ZeroMQReceiver.scala | 4 ++-- tools/pom.xml | 10 +++++----- yarn/pom.xml | 8 ++++---- 22 files changed, 114 insertions(+), 107 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/assembly/pom.xml b/assembly/pom.xml index 8103534796..c2cda41c6d 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -41,27 +41,27 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} org.apache.spark - spark-repl_${scala-short.version} + spark-repl_2.10 ${project.version} org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} @@ -79,7 +79,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala-short.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + ${project.build.directory}/scala-2.10/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar *:* @@ -128,7 +128,7 @@ org.apache.spark - spark-yarn_${scala-short.version} + spark-yarn_2.10 ${project.version} diff --git a/bagel/pom.xml b/bagel/pom.xml index 461e76a753..0f550d70d6 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -43,18 +43,18 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/core/pom.xml b/core/pom.xml index af605e1837..71bf15c491 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -86,7 +86,7 @@ com.twitter - chill_${scala-short.version} + chill_2.10 0.3.1 @@ -96,11 +96,11 @@ com.typesafe.akka - akka-remote_${scala-short.version} + akka-remote_2.10 com.typesafe.akka - akka-slf4j_${scala-short.version} + akka-slf4j_2.10 org.scala-lang @@ -108,7 +108,7 @@ net.liftweb - lift-json_${scala-short.version} + lift-json_2.10 it.unimi.dsi @@ -120,7 +120,7 @@ com.github.scala-incubator.io - scala-io-file_${scala-short.version} + scala-io-file_2.10 org.apache.mesos @@ -166,12 +166,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -191,8 +191,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 88a7f24884..d36e1b13a6 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -72,12 +72,11 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. private def askTracker(message: Any): Any = { try { - val future = if (trackerActor.isLeft ) { - trackerActor.left.get.ask(message)(timeout) - } else { - trackerActor.right.get.ask(message)(timeout) + val future = trackerActor match { + case Left(a: ActorRef) => a.ask(message)(timeout) + case Right(b: ActorSelection) => b.ask(message)(timeout) } - return Await.result(future, timeout) + Await.result(future, timeout) } catch { case e: Exception => throw new SparkException("Error communicating with MapOutputTracker", e) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 62b608c088..bcec41c439 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -17,11 +17,11 @@ package org.apache.spark -import org.apache.spark.util.Utils -import org.apache.spark.rdd.RDD - import scala.reflect.ClassTag +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. 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 a38e32b339..6c18a3c245 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 @@ -17,23 +17,19 @@ package org.apache.spark.deploy.worker.ui -import akka.actor.ActorRef -import akka.util.Timeout +import java.io.File import scala.concurrent.duration._ -import java.io.{FileInputStream, File} - +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.{Logging} -import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.{JettyUtils, UIUtils} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils +import org.eclipse.jetty.server.{Handler, Server} /** * Web UI server for the standalone worker. diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 99ea6e8ee8..a712ef1c27 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -17,16 +17,13 @@ package org.apache.spark.rdd +import java.io.IOException + import scala.reflect.ClassTag + +import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.io.{NullWritable, BytesWritable} -import org.apache.hadoop.util.ReflectionUtils -import org.apache.hadoop.fs.Path -import java.io.{File, IOException, EOFException} -import java.text.NumberFormat private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {} diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index e72f86fb13..8df8718f3b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.sql.{Connection, ResultSet} import scala.reflect.ClassTag + import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} import org.apache.spark.util.NextIterator diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala index eb3b19907d..8d7c288593 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, TaskContext} + private[spark] class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) extends RDD[U](prev) { diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index 4a465840c6..d5691f2267 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{RangePartitioner, Logging} import scala.reflect.ClassTag +import org.apache.spark.{Logging, RangePartitioner} + /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 1d109a2496..3682c84598 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -17,9 +17,10 @@ package org.apache.spark.rdd -import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, + SparkEnv, TaskContext} private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx 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 e5de16fc01..e05b842476 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -157,10 +157,9 @@ private[spark] class BlockManagerMaster(var driverActor : Either[ActorRef, Actor while (attempts < AKKA_RETRY_ATTEMPTS) { attempts += 1 try { - val future = if (driverActor.isLeft ) { - driverActor.left.get.ask(message)(timeout) - } else { - driverActor.right.get.ask(message)(timeout) + val future = driverActor match { + case Left(a: ActorRef) => a.ask(message)(timeout) + case Right(b: ActorSelection) => b.ask(message)(timeout) } val result = Await.result(future, timeout) if (result == null) { diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala index 69519860c6..bf71882ef7 100644 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala @@ -1,5 +1,18 @@ -/** - * Copyright (C) 2009-2013 Typesafe Inc. +/* + * 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. */ // Must be in akka.actor package as ActorSystemImpl is protected[akka]. diff --git a/examples/pom.xml b/examples/pom.xml index 464ad82e33..97f6dfea66 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -49,25 +49,25 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} provided org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} provided org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} provided org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} provided @@ -88,7 +88,7 @@ org.apache.kafka - kafka_2.9.2 + kafka_2.10 0.8.0-beta1 @@ -107,17 +107,17 @@ com.twitter - algebird-core_${scala-short.version} + algebird-core_2.10 0.1.11 org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -166,8 +166,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/mllib/pom.xml b/mllib/pom.xml index fce5b19bb2..228f8c029b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -34,7 +34,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -48,12 +48,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -63,8 +63,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/pom.xml b/pom.xml index 4c11459088..979fd0c287 100644 --- a/pom.xml +++ b/pom.xml @@ -101,7 +101,6 @@ 1.6 - 2.10 2.10.3 0.13.0 2.2.3 @@ -205,7 +204,7 @@ com.twitter - chill_${scala-short.version} + chill_2.10 0.3.1 @@ -215,7 +214,7 @@ com.typesafe.akka - akka-actor_${scala-short.version} + akka-actor_2.10 ${akka.version} @@ -226,7 +225,7 @@ com.typesafe.akka - akka-remote_${scala-short.version} + akka-remote_2.10 ${akka.version} @@ -237,7 +236,7 @@ com.typesafe.akka - akka-slf4j_${scala-short.version} + akka-slf4j_2.10 ${akka.version} @@ -258,7 +257,7 @@ com.github.scala-incubator.io - scala-io-file_${scala-short.version} + scala-io-file_2.10 0.4.1 @@ -279,7 +278,7 @@ net.liftweb - lift-json_${scala-short.version} + lift-json_2.10 2.5.1 @@ -335,7 +334,7 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 1.9.1 test @@ -358,7 +357,7 @@ org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 1.10.0 test diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index f8a17d93cc..c2a4efa36c 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -40,18 +40,18 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} runtime org.apache.spark - spark-repl_${scala-short.version} + spark-repl_2.10 ${project.version} runtime diff --git a/repl/pom.xml b/repl/pom.xml index 2f27e76760..bf06d730d4 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -39,18 +39,18 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-bagel_${scala-short.version} + spark-bagel_2.10 ${project.version} runtime org.apache.spark - spark-mllib_${scala-short.version} + spark-mllib_2.10 ${project.version} runtime @@ -78,18 +78,18 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/streaming/pom.xml b/streaming/pom.xml index ff95591b9f..298bc83161 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -48,7 +48,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -61,8 +61,8 @@ 1.9.11 - org.apache.kafka - kafka_2.9.2 + com.sksamuel.kafka + kafka_2.10 0.8.0-beta1 @@ -111,7 +111,7 @@ com.typesafe.akka - akka-zeromq_${scala-short.version} + akka-zeromq_2.10 ${akka.version} @@ -122,12 +122,12 @@ org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test org.scalacheck - scalacheck_${scala-short.version} + scalacheck_2.10 test @@ -151,8 +151,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.scalatest diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala index ce8c56fa8a..f164d516b0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala @@ -17,14 +17,14 @@ package org.apache.spark.streaming.receivers +import scala.reflect.ClassTag + import akka.actor.Actor import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging -import scala.reflect.ClassTag - /** * A receiver to subscribe to ZeroMQ stream. */ diff --git a/tools/pom.xml b/tools/pom.xml index 353d201f90..2bad494fc5 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -33,24 +33,24 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} org.apache.spark - spark-streaming_${scala-short.version} + spark-streaming_2.10 ${project.version} org.scalatest - scalatest_${scala-short.version} + scalatest_2.10 test - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins diff --git a/yarn/pom.xml b/yarn/pom.xml index 5cf81575c9..d18ac3736d 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -33,7 +33,7 @@ org.apache.spark - spark-core_${scala-short.version} + spark-core_2.10 ${project.version} @@ -63,7 +63,7 @@ org.scalatest - scalatest_2.9.3 + scalatest_2.10 test @@ -74,8 +74,8 @@ - target/scala-${scala-short.version}/classes - target/scala-${scala-short.version}/test-classes + target/scala-2.10/classes + target/scala-2.10/test-classes org.apache.maven.plugins -- cgit v1.2.3 From 5013fb64b27e46bbc5daf4f06fdc70938c06cf29 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 00:38:16 -0800 Subject: Expose numPartitions parameter in JavaPairRDD.sortByKey() This change make Java and Scala API on sortByKey() the same. --- .../main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 2142fd7327..a191dfd1dc 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -583,12 +583,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { + def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = sortByKey(comp, ascending, rdd.partitions.size) + + /** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ + def sortByKey(comp: Comparator[K], ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = { class KeyOrdering(val a: K) extends Ordered[K] { override def compare(b: K) = comp.compare(a, b) } implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) - fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) } /** -- cgit v1.2.3 From c82d4f079bf84da06801419defe644f574d0b8d1 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 01:04:52 -0800 Subject: Use braces to shorten the line. --- core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index a191dfd1dc..36bd3e673f 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -583,7 +583,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = sortByKey(comp, ascending, rdd.partitions.size) + def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { + sortByKey(comp, ascending, rdd.partitions.size) + } /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling -- cgit v1.2.3 From e85af507671d417724c28ee2db499fc019feb1d8 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 11:01:56 -0800 Subject: Leave default value of numPartitions to Scala code. --- .../src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 36bd3e673f..2d2b3847de 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -584,7 +584,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * order of the keys). */ def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { - sortByKey(comp, ascending, rdd.partitions.size) + // numPartitions should never be negative in practice so we can use -1 here to indicate that + // we want to use implementation's default value. + sortByKey(comp, ascending, -1) } /** @@ -598,7 +600,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif override def compare(b: K) = comp.compare(a, b) } implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) - fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) + if (numPartitions < 0) { + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) + } else { + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) + } } /** -- cgit v1.2.3 From 0b494f7db47cf1de35aaed046f21bbb3592c3d97 Mon Sep 17 00:00:00 2001 From: Binh Nguyen Date: Tue, 10 Dec 2013 11:17:52 -0800 Subject: Hook directly to Scala API --- .../main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 2d2b3847de..d0ca289a6c 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -584,9 +584,11 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif * order of the keys). */ def sortByKey(comp: Comparator[K], ascending: Boolean): JavaPairRDD[K, V] = { - // numPartitions should never be negative in practice so we can use -1 here to indicate that - // we want to use implementation's default value. - sortByKey(comp, ascending, -1) + class KeyOrdering(val a: K) extends Ordered[K] { + override def compare(b: K) = comp.compare(a, b) + } + implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) } /** @@ -600,11 +602,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif override def compare(b: K) = comp.compare(a, b) } implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) - if (numPartitions < 0) { - fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending)) - } else { - fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) - } + fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions)) } /** -- cgit v1.2.3 From a854cc536d79fc2c503d70bf8b6999d74f946876 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 13 Dec 2013 15:19:51 +0530 Subject: Review comments on the PR for scala 2.10 migration. --- .../main/scala/org/apache/spark/SparkContext.scala | 3 +-- .../org/apache/spark/api/java/JavaPairRDD.scala | 22 ++++++++-------------- .../org/apache/spark/api/java/JavaRDDLike.scala | 9 +++------ .../apache/spark/api/java/JavaSparkContext.scala | 15 +++++---------- .../spark/api/java/function/FlatMapFunction.scala | 2 +- .../org/apache/spark/api/python/PythonRDD.scala | 4 ++-- .../org/apache/spark/deploy/ExecutorState.scala | 3 +-- .../org/apache/spark/deploy/client/Client.scala | 2 +- .../scala/org/apache/spark/rdd/CartesianRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/EmptyRDD.scala | 2 +- .../org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- .../org/apache/spark/scheduler/TaskLocality.scala | 4 +--- .../org/apache/spark/storage/BlockManager.scala | 1 - project/SparkBuild.scala | 6 +++--- .../org/apache/spark/repl/SparkExprTyper.scala | 1 - 15 files changed, 29 insertions(+), 49 deletions(-) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f3ce4c879d..a0f794edfd 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -26,8 +26,7 @@ import scala.collection.Map import scala.collection.generic.Growable import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap -import scala.reflect.{ ClassTag, classTag} -import scala.util.DynamicVariable +import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index eeea0eddb1..1167c12022 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -158,8 +158,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner): JavaPairRDD[K, C] = { - implicit val cm: ClassTag[C] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] fromRDD(rdd.combineByKey( createCombiner, mergeValue, @@ -323,8 +322,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = { - implicit val cm: ClassTag[C] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd))) } @@ -415,8 +413,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K * this also retains the original RDD's partitioning. */ def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = { - implicit val cm: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.mapValues(f)) } @@ -427,8 +424,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ def fn = (x: V) => f.apply(x).asScala - implicit val cm: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] fromRDD(rdd.flatMapValues(fn)) } @@ -609,8 +605,8 @@ object JavaPairRDD { rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList _) def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassTag[K], - vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V], - Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd) + .mapValues((x: (Seq[V], Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2))) def cogroupResult2ToJava[W1, W2, K, V](rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))])(implicit kcm: ClassTag[K]) : RDD[(K, (JList[V], JList[W1], @@ -627,10 +623,8 @@ object JavaPairRDD { /** Convert a JavaRDD of key-value pairs to JavaPairRDD. */ def fromJavaRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] = { - implicit val cmk: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val cmv: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val cmk: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val cmv: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] new JavaPairRDD[K, V](rdd.rdd) } 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 c9c13f7a26..9e912d3adb 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 @@ -159,16 +159,14 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * elements (a, b) where a is in `this` and b is in `other`. */ def cartesian[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = - JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag, - other.classTag) + JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag, other.classTag) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val vcm: ClassTag[JList[T]] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm) @@ -179,8 +177,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * mapping to that key. */ def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { - implicit val kcm: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val vcm: ClassTag[JList[T]] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]] JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index d5b77357a2..acf328aa6a 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -83,8 +83,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } @@ -95,10 +94,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** Distribute a local Scala collection to form an RDD. */ def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]], numSlices: Int) : JavaPairRDD[K, V] = { - implicit val kcm: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val vcm: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] + implicit val vcm: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)) } @@ -154,8 +151,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path, minSplits)(cm) } @@ -167,8 +163,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * that there's very little effort required to save arbitrary objects. */ def objectFile[T](path: String): JavaRDD[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] sc.objectFile(path)(cm) } diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala index ed8fea97fc..bdb01f7670 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala @@ -23,5 +23,5 @@ import scala.reflect.ClassTag * A function that returns zero or more output records from each input record. */ abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]] { - def elementType() : ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]] + def elementType(): ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]] } 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 2bf7ac256e..a659cc06c2 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 @@ -148,7 +148,7 @@ private[spark] class PythonRDD[T: ClassTag]( case eof: EOFException => { throw new SparkException("Python worker exited unexpectedly (crashed)", eof) } - case e : Throwable => throw e + case e: Throwable => throw e } } @@ -200,7 +200,7 @@ private[spark] object PythonRDD { } } catch { case eof: EOFException => {} - case e : Throwable => throw e + case e: Throwable => throw e } JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala index 72f84c99fc..37dfa7fec0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala @@ -17,8 +17,7 @@ package org.apache.spark.deploy -private[spark] object ExecutorState - extends Enumeration { +private[spark] object ExecutorState extends Enumeration { val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value 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 d0d65ca72a..4d95efa73a 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 @@ -127,7 +127,7 @@ private[spark] class Client( val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) listener.executorAdded(fullId, workerId, hostPort, cores, memory) - + case ExecutorUpdated(id, state, message, exitStatus) => val fullId = appId + "/" + id val messageText = message.map(s => " (" + s + ")").getOrElse("") diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index d2a3d60965..87b950ba43 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -44,7 +44,7 @@ class CartesianPartition( } private[spark] -class CartesianRDD[T: ClassTag, U:ClassTag]( +class CartesianRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1 : RDD[T], var rdd2 : RDD[U]) diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index 99c34c6cc5..a84e5f9fd8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -17,9 +17,9 @@ package org.apache.spark.rdd -import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext} import scala.reflect.ClassTag +import org.apache.spark.{Partition, SparkContext, TaskContext} /** * An RDD that is empty, i.e. has no element in it. diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 0c2a051a42..48168e152e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -25,7 +25,7 @@ import java.util.{HashMap => JHashMap} import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ -import scala.reflect.{ ClassTag, classTag} +import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.mapred._ import org.apache.hadoop.io.compress.CompressionCodec diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index d31a5d5177..35de13c385 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -18,9 +18,7 @@ package org.apache.spark.scheduler -private[spark] object TaskLocality - extends Enumeration -{ +private[spark] object TaskLocality extends Enumeration { // process local is expected to be used ONLY within tasksetmanager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 7e721a49a5..19a025a329 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -865,7 +865,6 @@ private[spark] object BlockManager extends Logging { } def getHeartBeatFrequencyFromSystemProperties: Long = - System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4 def getDisableHeartBeatsForTesting: Boolean = diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c322cbd6b5..441dcc18fb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -84,9 +84,9 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } - if(isNewHadoop && isYarnEnabled) { - println("""Yarn with Hadoop version 2.2.x is not yet expected to work. - Please set env SPARK_HADOOP_VERSION to appropriate version or set SPARK_YARN to false.""") + if (isNewHadoop && isYarnEnabled) { + println( """Yarn with Hadoop version 2.2.x is not yet expected to work. + Please set env SPARK_HADOOP_VERSION to appropriate version or set SPARK_YARN to false.""") throw new Exception("Yarn with Hadoop version 2.2.x is not yet expected to work.") } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index 85b0978c81..b2e1df173e 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -12,7 +12,6 @@ import scala.reflect.internal.util.BatchSourceFile import scala.tools.nsc.ast.parser.Tokens.EOF import org.apache.spark.Logging -import scala.Some trait SparkExprTyper extends Logging { val repl: SparkIMain -- cgit v1.2.3 From 1ae3c0fc5e0273319d0e9497268a4c9854aae2e0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 14 Dec 2013 10:44:24 +0530 Subject: Added a comment about ActorRef and ActorSelection difference. --- core/src/main/scala/org/apache/spark/MapOutputTracker.scala | 7 +++++++ 1 file changed, 7 insertions(+) (limited to 'core/src/main/scala/org/apache') diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index fbda11f578..10fae5af9f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -72,6 +72,13 @@ private[spark] class MapOutputTracker extends Logging { // throw a SparkException if this fails. private def askTracker(message: Any): Any = { try { + /* + The difference between ActorRef and ActorSelection is well explained here: + http://doc.akka.io/docs/akka/2.2.3/project/migration-guide-2.1.x-2.2.x.html#Use_actorSelection_instead_of_actorFor + In spark a map output tracker can be either started on Driver where it is created which + is an ActorRef or it can be on executor from where it is looked up which is an + actorSelection. + */ val future = trackerActor match { case Left(a: ActorRef) => a.ask(message)(timeout) case Right(b: ActorSelection) => b.ask(message)(timeout) -- cgit v1.2.3 From 8a397a959bf0b68f7d10fa57665225e0c2b5d03a Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Mon, 16 Dec 2013 12:07:39 -0500 Subject: Track task value serialisation time in TaskMetrics. --- .../scala/org/apache/spark/executor/Executor.scala | 12 ++++++++---- .../org/apache/spark/executor/TaskMetrics.scala | 5 +++++ .../org/apache/spark/scheduler/TaskResult.scala | 20 ++++++++++---------- .../cluster/ClusterTaskSetManagerSuite.scala | 4 +++- 4 files changed, 26 insertions(+), 15 deletions(-) (limited to 'core/src/main/scala/org/apache') 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..02ad64d070 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 objectSer = SparkEnv.get.serializer.newInstance() + val beforeSerialization = System.currentTimeMillis() + val valueBytes = objectSer.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.serializationTime = (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..c036866afd 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -43,6 +43,11 @@ class TaskMetrics extends Serializable { */ var jvmGCTime: Long = _ + /** + * Amount of time spent serializing the result of the task + */ + var serializationTime: Long = _ + /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here */ 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..4e00bc8271 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 objectSer = SparkEnv.get.serializer.newInstance() + return objectSer.deserialize(valueBytes) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index b97f2b19b5..788cbb81bf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -313,6 +313,8 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo } def createTaskResult(id: Int): DirectTaskResult[Int] = { - new DirectTaskResult[Int](id, mutable.Map.empty, new TaskMetrics) + val objectSer = SparkEnv.get.serializer.newInstance() + new DirectTaskResult[Int](objectSer.serialize(id), mutable.Map.empty, + new TaskMetrics) } } -- cgit v1.2.3 From 882d544856c61573cdd6124e921f700d580d170d Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Mon, 16 Dec 2013 13:27:03 -0500 Subject: UI to display serialisation time of a stage. --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 ++++++ 1 file changed, 6 insertions(+) (limited to 'core/src/main/scala/org/apache') 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 69f9446bab..81651bdd20 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 @@ -101,6 +101,11 @@ private[spark] class StagePage(parent: JobProgressUI) { None } else { + val serializationTimes = validTasks.map{case (info, metrics, exception) => + metrics.get.serializationTime.toDouble} + val serializationQuantiles = "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, -- cgit v1.2.3 From 963d6f065a763c2b94529bbd3ac4326e190bb2d7 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Mon, 16 Dec 2013 23:14:52 -0500 Subject: Incorporate pwendell's code review suggestions. --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 6 +++--- core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala | 4 ++-- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- .../apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala | 3 +-- 4 files changed, 8 insertions(+), 9 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 02ad64d070..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,9 +222,9 @@ private[spark] class Executor( return } - val objectSer = SparkEnv.get.serializer.newInstance() + val resultSer = SparkEnv.get.serializer.newInstance() val beforeSerialization = System.currentTimeMillis() - val valueBytes = objectSer.serialize(value) + val valueBytes = resultSer.serialize(value) val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { @@ -232,7 +232,7 @@ private[spark] class Executor( m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt m.jvmGCTime = gcTime - startGCTime - m.serializationTime = (afterSerialization - beforeSerialization).toInt + m.resultSerializationTime = (afterSerialization - beforeSerialization).toInt } val accumUpdates = Accumulators.values 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 c036866afd..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,9 +44,9 @@ class TaskMetrics extends Serializable { var jvmGCTime: Long = _ /** - * Amount of time spent serializing the result of the task + * Amount of time spent serializing the task result */ - var serializationTime: Long = _ + var resultSerializationTime: Long = _ /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here 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 81651bdd20..2f06efa66e 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 @@ -102,8 +102,8 @@ private[spark] class StagePage(parent: JobProgressUI) { } else { val serializationTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.serializationTime.toDouble} - val serializationQuantiles = "Serialization Time" +: Distribution(serializationTimes).get.getQuantiles().map( + 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) => diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala index 788cbb81bf..2476ab5c19 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala @@ -314,7 +314,6 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo def createTaskResult(id: Int): DirectTaskResult[Int] = { val objectSer = SparkEnv.get.serializer.newInstance() - new DirectTaskResult[Int](objectSer.serialize(id), mutable.Map.empty, - new TaskMetrics) + new DirectTaskResult[Int](objectSer.serialize(id), mutable.Map.empty, new TaskMetrics) } } -- cgit v1.2.3 From b2f0329511f3caaf473cf300792690703a300a22 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Tue, 17 Dec 2013 00:18:46 -0500 Subject: Missed a spot; had an objectSer here too. --- core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 4e00bc8271..e80cc6b0f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -73,7 +73,7 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long } def value(): T = { - val objectSer = SparkEnv.get.serializer.newInstance() - return objectSer.deserialize(valueBytes) + val resultSer = SparkEnv.get.serializer.newInstance() + return resultSer.deserialize(valueBytes) } } -- cgit v1.2.3 From 9a6864d016f0f923c885ca2a4977104cd19ded4f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 17 Dec 2013 18:44:39 -0800 Subject: Fixed a performance problem in RDD.top and BoundedPriorityQueue (size in BoundedPriority was actually traversing the entire queue to calculate the size, resulting in bad performance in insertion). --- core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala | 2 ++ 1 file changed, 2 insertions(+) (limited to 'core/src/main/scala/org/apache') 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 -- cgit v1.2.3 From d3b1af4b6c7766bbf7a09ee6d5c1b13eda6b098f Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Wed, 18 Dec 2013 14:25:56 -0500 Subject: Add a serialisation time column to the StagePage. --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) (limited to 'core/src/main/scala/org/apache') 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 2f06efa66e..996e1b4d1a 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") @@ -189,6 +189,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("") @@ -216,6 +217,9 @@ private[spark] class StagePage(parent: JobProgressUI) { + {if (shuffleRead) {
ReleaseVersion code
CDH 4.X.X (YARN mode)2.0.0-chd4.X.X
CDH 4.X.X2.0.0-mr1-chd4.X.X
CDH 4.X.X (YARN mode)2.0.0-cdh4.X.X
CDH 4.X.X2.0.0-mr1-cdh4.X.X
CDH 3u60.20.2-cdh3u6
CDH 3u50.20.2-cdh3u5
CDH 3u40.20.2-cdh3u4
spark.akka.pausesspark.akka.heartbeat.pauses 60 Acceptable heart beat pause in seconds for akka, tune this if you expect GC pauses or network delays (reconnections) etc. @@ -298,7 +298,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval 5 - A larger interval value in seconds reduces network overhead and a smaller value might be more informative for akka's failure detector. Tune this in combination of `spark.akka.pauses` and `spark.akka.failure-detector.threshold` if you need to. + A larger interval value in seconds reduces network overhead and a smaller value might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to.
{if (gcTime > 0) parent.formatDuration(gcTime) else ""} + {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + {shuffleReadReadable} -- cgit v1.2.3 From af0cd6bd27dda73b326bcb6a66addceadebf5e54 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 18 Dec 2013 11:40:07 -0800 Subject: Add collectPartition to JavaRDD interface. Also remove takePartition from PythonRDD and use collectPartition in rdd.py. --- .../org/apache/spark/api/java/JavaRDDLike.scala | 11 ++++++++- .../org/apache/spark/api/python/PythonRDD.scala | 4 ---- .../test/scala/org/apache/spark/JavaAPISuite.java | 28 ++++++++++++++++++++++ python/pyspark/context.py | 3 --- python/pyspark/rdd.py | 2 +- 5 files changed, 39 insertions(+), 9 deletions(-) (limited to 'core/src/main/scala/org/apache') 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..1d71875ed1 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 @@ -26,7 +26,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{SparkContext, Partition, TaskContext} -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, PartitionPruningRDD} import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import org.apache.spark.partial.{PartialResult, BoundedDouble} @@ -244,6 +244,15 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Return an array that contains all of the elements in a specific partition of this RDD. + */ + def collectPartition(partitionId: Int): JList[T] = { + import scala.collection.JavaConversions._ + val partition = new PartitionPruningRDD[T](rdd, _ == partitionId) + new java.util.ArrayList(partition.collect().toSeq) + } + /** * Reduces the elements of this RDD using the specified commutative and associative binary operator. */ 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/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 4234f6eac7..2862ed3019 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -897,4 +897,32 @@ public class JavaAPISuite implements Serializable { new Tuple2(0, 4)), rdd3.collect()); } + + @Test + public void collectPartition() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); + + JavaPairRDD rdd2 = rdd1.map(new PairFunction() { + @Override + public Tuple2 call(Integer i) throws Exception { + return new Tuple2(i, i % 2); + } + }); + + Assert.assertEquals(Arrays.asList(1, 2), rdd1.collectPartition(0)); + Assert.assertEquals(Arrays.asList(3, 4), rdd1.collectPartition(1)); + Assert.assertEquals(Arrays.asList(5, 6, 7), rdd1.collectPartition(2)); + + Assert.assertEquals(Arrays.asList(new Tuple2(1, 1), + new Tuple2(2, 0)), + rdd2.collectPartition(0)); + Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), + new Tuple2(4, 0)), + rdd2.collectPartition(1)); + Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), + new Tuple2(6, 0), + new Tuple2(7, 1)), + rdd2.collectPartition(2)); + } + } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index cbd41e58c4..0604f6836c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -43,7 +43,6 @@ class SparkContext(object): _gateway = None _jvm = None _writeToFile = None - _takePartition = None _next_accum_id = 0 _active_spark_context = None _lock = Lock() @@ -134,8 +133,6 @@ class SparkContext(object): SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = \ SparkContext._jvm.PythonRDD.writeToFile - SparkContext._takePartition = \ - SparkContext._jvm.PythonRDD.takePartition if instance: if SparkContext._active_spark_context and SparkContext._active_spark_context != instance: diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 61720dcf1a..d81b7c90c1 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -577,7 +577,7 @@ class RDD(object): mapped = self.mapPartitions(takeUpToNum) items = [] for partition in range(mapped._jrdd.splits().size()): - iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition) + iterator = mapped._jrdd.collectPartition(partition).iterator() items.extend(mapped._collect_iterator_through_file(iterator)) if len(items) >= num: break -- cgit v1.2.3 From 293a0af5a1def95e47d9188f42957083f5adf3b8 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 18 Dec 2013 18:51:02 -0800 Subject: In experimental clusters we've observed that a 10 second timeout was insufficient, despite having a low number of nodes and relatively small workload (16 nodes, <1.5 TB data). This would cause an entire job to fail at the beginning of the reduce phase. There is no particular reason for this value to be small as a timeout should only occur in an exceptional situation. Also centralized the reading of spark.akka.askTimeout to AkkaUtils (surely this can later be cleaned up to use Typesafe). Finally, deleted some lurking implicits. If anyone can think of a reason they should still be there, please let me know. --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 5 ++--- .../scala/org/apache/spark/deploy/client/Client.scala | 6 +++--- .../scala/org/apache/spark/deploy/master/Master.scala | 17 ++++++----------- .../apache/spark/deploy/master/ui/ApplicationPage.scala | 14 +++++--------- .../org/apache/spark/deploy/master/ui/IndexPage.scala | 16 ++++++---------- .../org/apache/spark/deploy/master/ui/MasterWebUI.scala | 10 +++------- .../org/apache/spark/deploy/worker/ui/IndexPage.scala | 4 ++-- .../org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 10 +++------- .../cluster/CoarseGrainedSchedulerBackend.scala | 10 ++++------ .../org/apache/spark/storage/BlockManagerMaster.scala | 7 ++++--- .../apache/spark/storage/BlockManagerMasterActor.scala | 11 ++++------- .../org/apache/spark/ui/storage/BlockManagerUI.scala | 3 --- .../main/scala/org/apache/spark/util/AkkaUtils.scala | 6 ++++++ 13 files changed, 48 insertions(+), 71 deletions(-) (limited to 'core/src/main/scala/org/apache') 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/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..7b2b1c3327 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 @@ -537,12 +534,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/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/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/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") + } } -- cgit v1.2.3 From a76f53416c0267e8a9816ee4d22fe8e838c4c319 Mon Sep 17 00:00:00 2001 From: Nick Pentreath Date: Thu, 19 Dec 2013 14:38:20 +0200 Subject: Add toString to Java RDD, and __repr__ to Python RDD --- core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala | 2 ++ python/pyspark/rdd.py | 3 +++ 2 files changed, 5 insertions(+) (limited to 'core/src/main/scala/org/apache') 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/python/pyspark/rdd.py b/python/pyspark/rdd.py index 61720dcf1a..7cbc66d3c9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -54,6 +54,9 @@ class RDD(object): self.ctx = ctx self._jrdd_deserializer = jrdd_deserializer + def __repr__(self): + return self._jrdd.toString() + @property def context(self): """ -- cgit v1.2.3 From d3234f9726db3917af4688ba70933938b078b0bd Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 19 Dec 2013 11:40:34 -0800 Subject: Make collectPartitions take an array of partitions Change the implementation to use runJob instead of PartitionPruningRDD. Also update the unit tests and the python take implementation to use the new interface. --- .../scala/org/apache/spark/api/java/JavaRDDLike.scala | 8 ++++---- .../src/test/scala/org/apache/spark/JavaAPISuite.java | 19 ++++++++++++------- python/pyspark/rdd.py | 7 ++++++- 3 files changed, 22 insertions(+), 12 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 1d71875ed1..458d9dcbc3 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 @@ -26,7 +26,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{SparkContext, Partition, TaskContext} -import org.apache.spark.rdd.{RDD, PartitionPruningRDD} +import org.apache.spark.rdd.RDD import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _} import org.apache.spark.partial.{PartialResult, BoundedDouble} @@ -247,10 +247,10 @@ 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 collectPartition(partitionId: Int): JList[T] = { + def collectPartitions(partitionIds: Array[Int]): Array[JList[T]] = { import scala.collection.JavaConversions._ - val partition = new PartitionPruningRDD[T](rdd, _ == partitionId) - new java.util.ArrayList(partition.collect().toSeq) + val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds, true) + res.map(x => new java.util.ArrayList(x.toSeq)).toArray } /** diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java index 2862ed3019..79913dc718 100644 --- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java @@ -899,7 +899,7 @@ public class JavaAPISuite implements Serializable { } @Test - public void collectPartition() { + public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); JavaPairRDD rdd2 = rdd1.map(new PairFunction() { @@ -909,20 +909,25 @@ public class JavaAPISuite implements Serializable { } }); - Assert.assertEquals(Arrays.asList(1, 2), rdd1.collectPartition(0)); - Assert.assertEquals(Arrays.asList(3, 4), rdd1.collectPartition(1)); - Assert.assertEquals(Arrays.asList(5, 6, 7), rdd1.collectPartition(2)); + List[] parts = rdd1.collectPartitions(new int[] {0}); + Assert.assertEquals(Arrays.asList(1, 2), parts[0]); + + parts = rdd1.collectPartitions(new int[] {1, 2}); + Assert.assertEquals(Arrays.asList(3, 4), parts[0]); + Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); Assert.assertEquals(Arrays.asList(new Tuple2(1, 1), new Tuple2(2, 0)), - rdd2.collectPartition(0)); + rdd2.collectPartitions(new int[] {0})[0]); + + parts = rdd2.collectPartitions(new int[] {1, 2}); Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), new Tuple2(4, 0)), - rdd2.collectPartition(1)); + parts[0]); Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), new Tuple2(6, 0), new Tuple2(7, 1)), - rdd2.collectPartition(2)); + parts[1]); } } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d81b7c90c1..7015119551 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -576,8 +576,13 @@ class RDD(object): # Take only up to num elements from each partition we try mapped = self.mapPartitions(takeUpToNum) items = [] + # TODO(shivaram): Similar to the scala implementation, update the take + # method to scan multiple splits based on an estimate of how many elements + # we have per-split. for partition in range(mapped._jrdd.splits().size()): - iterator = mapped._jrdd.collectPartition(partition).iterator() + partitionsToTake = self.ctx._gateway.new_array(self.ctx._jvm.int, 1) + partitionsToTake[0] = partition + iterator = mapped._jrdd.collectPartitions(partitionsToTake)[0].iterator() items.extend(mapped._collect_iterator_through_file(iterator)) if len(items) >= num: break -- cgit v1.2.3 From 9cc3a6d3c0a64b80af77ae358c58d4b29b18c534 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 19 Dec 2013 11:49:17 -0800 Subject: Add comment explaining collectPartitions's use --- core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 2 ++ 1 file changed, 2 insertions(+) (limited to 'core/src/main/scala/org/apache') 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 458d9dcbc3..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 @@ -248,6 +248,8 @@ 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 -- cgit v1.2.3 From 1ab031eaff7fb2473adb3e909a7a969e9cd28b49 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Dec 2013 17:51:29 -0800 Subject: Extraordinarily minor code/comment cleanup --- .../org/apache/spark/deploy/master/SparkZooKeeperSession.scala | 10 +++++----- .../spark/deploy/master/ZooKeeperLeaderElectionAgent.scala | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) (limited to 'core/src/main/scala/org/apache') 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..5b957fcd5a 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", "") @@ -179,7 +179,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 -- cgit v1.2.3 From 4d74b899b7daff74054b70f38cddf7b38fe6c211 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Dec 2013 17:53:41 -0800 Subject: Remove firstApp from the standalone scheduler Master As a lonely child with no one to care for it... we had to put it down. --- .../src/main/scala/org/apache/spark/deploy/master/Master.scala | 10 ---------- 1 file changed, 10 deletions(-) (limited to 'core/src/main/scala/org/apache') 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 7b2b1c3327..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 @@ -61,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") @@ -441,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 } -- cgit v1.2.3 From 6613ab663d42f6b54fe823b06307b8a1005bbb6a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 19 Dec 2013 17:56:13 -0800 Subject: Fix compiler warning in SparkZooKeeperSession --- .../scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala | 1 + 1 file changed, 1 insertion(+) (limited to 'core/src/main/scala/org/apache') 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 5b957fcd5a..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 @@ -103,6 +103,7 @@ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) ext connectToZooKeeper() case KeeperState.Disconnected => logWarning("ZooKeeper disconnected, will retry...") + case s => // Do nothing } } } -- cgit v1.2.3