aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--assembly/README11
-rw-r--r--assembly/pom.xml24
-rw-r--r--bagel/pom.xml10
-rwxr-xr-xbin/stop-slaves.sh2
-rw-r--r--conf/fairscheduler.xml.template18
-rw-r--r--conf/metrics.properties.template50
-rw-r--r--core/pom.xml28
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java3
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java6
-rw-r--r--core/src/main/scala/org/apache/spark/Aggregator.scala49
-rw-r--r--core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/CacheManager.scala35
-rw-r--r--core/src/main/scala/org/apache/spark/FutureAction.scala250
-rw-r--r--core/src/main/scala/org/apache/spark/InterruptibleIterator.scala30
-rw-r--r--core/src/main/scala/org/apache/spark/MapOutputTracker.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/ShuffleFetcher.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala137
-rw-r--r--core/src/main/scala/org/apache/spark/SparkEnv.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/TaskContext.scala24
-rw-r--r--core/src/main/scala/org/apache/spark/TaskEndReason.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala25
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala34
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala420
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/Client.scala84
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala53
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala90
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala45
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala245
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala46
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala53
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala26
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala203
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala42
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala136
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala85
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala175
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/executor/Executor.scala146
-rw-r--r--core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala82
-rw-r--r--core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala22
-rw-r--r--core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala27
-rw-r--r--core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala122
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala26
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala174
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala (renamed from core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala)12
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala79
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/RDD.scala150
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala146
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala25
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala62
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Pool.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala)15
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala52
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala)8
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala)108
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala)2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala)2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala50
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala18
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Task.scala63
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala)2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala)2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala)2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala)5
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala94
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala238
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala26
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala124
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala)20
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala)22
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala196
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala32
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala30
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockException.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala24
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockId.scala96
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala155
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockMessage.scala38
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockStore.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/storage/DiskStore.scala39
-rw-r--r--core/src/main/scala/org/apache/spark/storage/MemoryStore.scala48
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageLevel.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageUtils.scala47
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/SparkUI.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/ui/UIUtils.scala39
-rw-r--r--core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala230
-rw-r--r--core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala36
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala39
-rw-r--r--core/src/test/scala/org/apache/spark/CacheManagerSuite.scala94
-rw-r--r--core/src/test/scala/org/apache/spark/CheckpointSuite.scala10
-rw-r--r--core/src/test/scala/org/apache/spark/DistributedSuite.scala29
-rw-r--r--core/src/test/scala/org/apache/spark/JavaAPISuite.java2
-rw-r--r--core/src/test/scala/org/apache/spark/JobCancellationSuite.scala177
-rw-r--r--core/src/test/scala/org/apache/spark/LocalSparkContext.scala10
-rw-r--r--core/src/test/scala/org/apache/spark/SharedSparkContext.scala6
-rw-r--r--core/src/test/scala/org/apache/spark/ThreadingSuite.scala45
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala93
-rw-r--r--core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala19
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala176
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala53
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala31
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala30
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala23
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala58
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala114
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala34
-rw-r--r--core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala21
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala114
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala102
-rw-r--r--core/src/test/scala/org/apache/spark/ui/UISuite.scala9
-rw-r--r--core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala154
-rw-r--r--core/src/test/scala/org/apache/spark/util/UtilsSuite.scala11
-rw-r--r--docker/README.md5
-rwxr-xr-xdocker/build22
-rw-r--r--docker/spark-test/README.md10
-rw-r--r--docker/spark-test/base/Dockerfile38
-rwxr-xr-xdocker/spark-test/build22
-rw-r--r--docker/spark-test/master/Dockerfile21
-rwxr-xr-xdocker/spark-test/master/default_cmd22
-rw-r--r--docker/spark-test/worker/Dockerfile22
-rwxr-xr-xdocker/spark-test/worker/default_cmd22
-rw-r--r--docs/_config.yml6
-rwxr-xr-xdocs/_layouts/global.html13
-rw-r--r--docs/building-with-maven.md6
-rw-r--r--docs/cluster-overview.md117
-rw-r--r--docs/configuration.md34
-rw-r--r--docs/contributing-to-spark.md24
-rw-r--r--docs/ec2-scripts.md8
-rw-r--r--docs/hadoop-third-party-distributions.md118
-rw-r--r--docs/hardware-provisioning.md4
-rw-r--r--docs/img/cluster-overview.pngbin0 -> 28011 bytes
-rw-r--r--docs/img/cluster-overview.pptxbin0 -> 51771 bytes
-rw-r--r--docs/img/spark-logo-hd.pngbin0 -> 13512 bytes
-rw-r--r--docs/index.md26
-rw-r--r--docs/job-scheduling.md168
-rw-r--r--docs/mllib-guide.md208
-rw-r--r--docs/monitoring.md70
-rw-r--r--docs/python-programming-guide.md14
-rw-r--r--docs/quick-start.md70
-rw-r--r--docs/running-on-mesos.md48
-rw-r--r--docs/running-on-yarn.md45
-rw-r--r--docs/scala-programming-guide.md4
-rw-r--r--docs/spark-standalone.md121
-rw-r--r--docs/streaming-programming-guide.md5
-rw-r--r--docs/tuning.md2
-rw-r--r--ec2/README2
-rw-r--r--ec2/deploy.generic/root/spark-ec2/ec2-variables.sh17
-rwxr-xr-xec2/spark_ec2.py121
-rw-r--r--examples/pom.xml46
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala2
-rwxr-xr-xmake-distribution.sh2
-rw-r--r--mllib/data/sample_svm_data.txt322
-rw-r--r--mllib/pom.xml10
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala13
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala199
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala4
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java85
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala75
-rw-r--r--pom.xml282
-rw-r--r--project/SparkBuild.scala36
-rw-r--r--project/project/SparkPluginBuild.scala19
-rw-r--r--python/pyspark/__init__.py5
-rw-r--r--python/pyspark/context.py11
-rw-r--r--python/pyspark/rdd.py89
-rw-r--r--python/pyspark/serializers.py4
-rw-r--r--python/pyspark/shell.py3
-rw-r--r--python/pyspark/storagelevel.py43
-rw-r--r--repl-bin/pom.xml22
-rw-r--r--repl/pom.xml34
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala2
-rwxr-xr-xrun-example8
-rwxr-xr-xspark-class12
-rw-r--r--streaming/pom.xml11
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala11
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala14
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala4
-rw-r--r--tools/pom.xml10
-rw-r--r--yarn/pom.xml72
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala131
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala2
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala172
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala31
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala66
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala13
-rw-r--r--yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala3
248 files changed, 8454 insertions, 2259 deletions
diff --git a/assembly/README b/assembly/README
index 6ee2a536d7..14a5ff8dfc 100644
--- a/assembly/README
+++ b/assembly/README
@@ -4,10 +4,9 @@ It creates a single tar.gz file that includes all needed dependency of the proje
except for org.apache.hadoop.* jars that are supposed to be available from the
deployed Hadoop cluster.
-This module is off by default to avoid spending extra time on top of repl-bin
-module. To activate it specify the profile in the command line
- -Passembly
+This module is off by default. To activate it specify the profile in the command line
+ -Pbigtop-dist
-In case you want to avoid building time-expensive repl-bin module, that shaders
-all the dependency into a big flat jar supplement maven command with
- -DnoExpensive
+If you need to build an assembly for a different version of Hadoop the
+hadoop-version system property needs to be set as in this example:
+ -Dhadoop.version=2.0.6-alpha
diff --git a/assembly/pom.xml b/assembly/pom.xml
index d19f44d292..09df8c1fd7 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -21,12 +21,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-assembly</artifactId>
+ <artifactId>spark-assembly_2.9.3</artifactId>
<name>Spark Project Assembly</name>
<url>http://spark.incubator.apache.org/</url>
@@ -41,27 +41,27 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel</artifactId>
+ <artifactId>spark-bagel_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib</artifactId>
+ <artifactId>spark-mllib_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl</artifactId>
+ <artifactId>spark-repl_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming</artifactId>
+ <artifactId>spark-streaming_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -104,7 +104,13 @@
</goals>
<configuration>
<transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
+ <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+ <resource>META-INF/services/org.apache.hadoop.fs.FileSystem</resource>
+ </transformer>
+ </transformers>
+ <transformers>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>reference.conf</resource>
</transformer>
@@ -122,7 +128,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn</artifactId>
+ <artifactId>spark-yarn_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
diff --git a/bagel/pom.xml b/bagel/pom.xml
index 51173c32b2..0e552c880f 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -21,12 +21,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel</artifactId>
+ <artifactId>spark-bagel_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project Bagel</name>
<url>http://spark.incubator.apache.org/</url>
@@ -34,7 +34,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -43,12 +43,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.9.3</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
diff --git a/bin/stop-slaves.sh b/bin/stop-slaves.sh
index 03e416a132..fcb8555d4e 100755
--- a/bin/stop-slaves.sh
+++ b/bin/stop-slaves.sh
@@ -17,8 +17,6 @@
# limitations under the License.
#
-# Starts the master on the machine this script is executed on.
-
bin=`dirname "$0"`
bin=`cd "$bin"; pwd`
diff --git a/conf/fairscheduler.xml.template b/conf/fairscheduler.xml.template
index 04a6b418dc..acf59e2a35 100644
--- a/conf/fairscheduler.xml.template
+++ b/conf/fairscheduler.xml.template
@@ -1,15 +1,13 @@
<?xml version="1.0"?>
<allocations>
-<pool name="production">
- <minShare>2</minShare>
- <weight>1</weight>
+ <pool name="production">
<schedulingMode>FAIR</schedulingMode>
-</pool>
-<pool name="test">
- <minShare>3</minShare>
- <weight>2</weight>
+ <weight>1</weight>
+ <minShare>2</minShare>
+ </pool>
+ <pool name="test">
<schedulingMode>FIFO</schedulingMode>
-</pool>
-<pool name="data">
-</pool>
+ <weight>2</weight>
+ <minShare>3</minShare>
+ </pool>
</allocations>
diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template
index 6c36f3cca4..ae10f615d1 100644
--- a/conf/metrics.properties.template
+++ b/conf/metrics.properties.template
@@ -31,7 +31,7 @@
# 1. To add a new sink, set the "class" option to a fully qualified class
# name (see examples below).
# 2. Some sinks involve a polling period. The minimum allowed polling period
-# is 1 second.
+# is 1 second.
# 3. Wild card properties can be overridden by more specific properties.
# For example, master.sink.console.period takes precedence over
# *.sink.console.period.
@@ -47,11 +47,45 @@
# instance master and applications. MetricsServlet may not be configured by self.
#
+## List of available sinks and their properties.
+
+# org.apache.spark.metrics.sink.ConsoleSink
+# Name: Default: Description:
+# period 10 Poll period
+# unit seconds Units of poll period
+
+# org.apache.spark.metrics.sink.CSVSink
+# Name: Default: Description:
+# period 10 Poll period
+# unit seconds Units of poll period
+# directory /tmp Where to store CSV files
+
+# org.apache.spark.metrics.sink.GangliaSink
+# Name: Default: Description:
+# host NONE Hostname or multicast group of Ganglia server
+# port NONE Port of Ganglia server(s)
+# period 10 Poll period
+# unit seconds Units of poll period
+# ttl 1 TTL of messages sent by Ganglia
+# mode multicast Ganglia network mode ('unicast' or 'mulitcast')
+
+# org.apache.spark.metrics.sink.JmxSink
+
+# org.apache.spark.metrics.sink.MetricsServlet
+# Name: Default: Description:
+# path VARIES* Path prefix from the web server root
+# sample false Whether to show entire set of samples for histograms ('false' or 'true')
+#
+# * Default path is /metrics/json for all instances except the master. The master has two paths:
+# /metrics/aplications/json # App information
+# /metrics/master/json # Master information
+
+## Examples
# Enable JmxSink for all instances by class name
-#*.sink.jmx.class=spark.metrics.sink.JmxSink
+#*.sink.jmx.class=org.apache.spark.metrics.sink.JmxSink
# Enable ConsoleSink for all instances by class name
-#*.sink.console.class=spark.metrics.sink.ConsoleSink
+#*.sink.console.class=org.apache.spark.metrics.sink.ConsoleSink
# Polling period for ConsoleSink
#*.sink.console.period=10
@@ -64,7 +98,7 @@
#master.sink.console.unit=seconds
# Enable CsvSink for all instances
-#*.sink.csv.class=spark.metrics.sink.CsvSink
+#*.sink.csv.class=org.apache.spark.metrics.sink.CsvSink
# Polling period for CsvSink
#*.sink.csv.period=1
@@ -80,11 +114,11 @@
#worker.sink.csv.unit=minutes
# Enable jvm source for instance master, worker, driver and executor
-#master.source.jvm.class=spark.metrics.source.JvmSource
+#master.source.jvm.class=org.apache.spark.metrics.source.JvmSource
-#worker.source.jvm.class=spark.metrics.source.JvmSource
+#worker.source.jvm.class=org.apache.spark.metrics.source.JvmSource
-#driver.source.jvm.class=spark.metrics.source.JvmSource
+#driver.source.jvm.class=org.apache.spark.metrics.source.JvmSource
-#executor.source.jvm.class=spark.metrics.source.JvmSource
+#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource
diff --git a/core/pom.xml b/core/pom.xml
index 5738b7406f..8621d257e5 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -21,12 +21,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project Core</name>
<url>http://spark.incubator.apache.org/</url>
@@ -37,6 +37,10 @@
<artifactId>hadoop-client</artifactId>
</dependency>
<dependency>
+ <groupId>net.java.dev.jets3t</groupId>
+ <artifactId>jets3t</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
</dependency>
@@ -45,6 +49,10 @@
<artifactId>avro-ipc</artifactId>
</dependency>
<dependency>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>zookeeper</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
</dependency>
@@ -147,18 +155,22 @@
<artifactId>metrics-json</artifactId>
</dependency>
<dependency>
+ <groupId>com.codahale.metrics</groupId>
+ <artifactId>metrics-ganglia</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.apache.derby</groupId>
<artifactId>derby</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -193,14 +205,14 @@
<configuration>
<exportAntProperties>true</exportAntProperties>
<tasks>
- <property name="spark.classpath" refid="maven.test.classpath"/>
- <property environment="env"/>
+ <property name="spark.classpath" refid="maven.test.classpath" />
+ <property environment="env" />
<fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">
<condition>
<not>
<or>
- <isset property="env.SCALA_HOME"/>
- <isset property="env.SCALA_LIBRARY_PATH"/>
+ <isset property="env.SCALA_HOME" />
+ <isset property="env.SCALA_LIBRARY_PATH" />
</or>
</not>
</condition>
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
index c4aa2669e0..8a09210245 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java
@@ -21,6 +21,7 @@ import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundByteHandlerAdapter;
+import org.apache.spark.storage.BlockId;
abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
@@ -33,7 +34,7 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
}
public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header);
- public abstract void handleError(String blockId);
+ public abstract void handleError(BlockId blockId);
@Override
public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) {
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
index d3d57a0255..cfd8132891 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
@@ -24,6 +24,7 @@ import io.netty.channel.ChannelHandlerContext;
import io.netty.channel.ChannelInboundMessageHandlerAdapter;
import io.netty.channel.DefaultFileRegion;
+import org.apache.spark.storage.BlockId;
class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
@@ -34,8 +35,9 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
}
@Override
- public void messageReceived(ChannelHandlerContext ctx, String blockId) {
- String path = pResolver.getAbsolutePath(blockId);
+ public void messageReceived(ChannelHandlerContext ctx, String blockIdString) {
+ BlockId blockId = BlockId.apply(blockIdString);
+ String path = pResolver.getAbsolutePath(blockId.name());
// if getFilePath returns null, close the channel
if (path == null) {
//ctx.close();
diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index 3ef402926e..1a2ec55876 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -17,43 +17,42 @@
package org.apache.spark
-import java.util.{HashMap => JHashMap}
+import org.apache.spark.util.AppendOnlyMap
-import scala.collection.JavaConversions._
-
-/** A set of functions used to aggregate data.
- *
- * @param createCombiner function to create the initial value of the aggregation.
- * @param mergeValue function to merge a new value into the aggregation result.
- * @param mergeCombiners function to merge outputs from multiple mergeValue function.
- */
+/**
+ * A set of functions used to aggregate data.
+ *
+ * @param createCombiner function to create the initial value of the aggregation.
+ * @param mergeValue function to merge a new value into the aggregation result.
+ * @param mergeCombiners function to merge outputs from multiple mergeValue function.
+ */
case class Aggregator[K, V, C] (
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C) {
def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]) : Iterator[(K, C)] = {
- val combiners = new JHashMap[K, C]
- for (kv <- iter) {
- val oldC = combiners.get(kv._1)
- if (oldC == null) {
- combiners.put(kv._1, createCombiner(kv._2))
- } else {
- combiners.put(kv._1, mergeValue(oldC, kv._2))
- }
+ val combiners = new AppendOnlyMap[K, C]
+ var kv: Product2[K, V] = null
+ val update = (hadValue: Boolean, oldValue: C) => {
+ if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2)
+ }
+ while (iter.hasNext) {
+ kv = iter.next()
+ combiners.changeValue(kv._1, update)
}
combiners.iterator
}
def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = {
- val combiners = new JHashMap[K, C]
- iter.foreach { case(k, c) =>
- val oldC = combiners.get(k)
- if (oldC == null) {
- combiners.put(k, c)
- } else {
- combiners.put(k, mergeCombiners(oldC, c))
- }
+ val combiners = new AppendOnlyMap[K, C]
+ var kc: (K, C) = null
+ val update = (hadValue: Boolean, oldValue: C) => {
+ if (hadValue) mergeCombiners(oldValue, kc._2) else kc._2
+ }
+ while (iter.hasNext) {
+ kc = iter.next()
+ combiners.changeValue(kc._1, update)
}
combiners.iterator
}
diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
index 908ff56a6b..d9ed572da6 100644
--- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala
@@ -22,13 +22,17 @@ import scala.collection.mutable.HashMap
import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics}
import org.apache.spark.serializer.Serializer
-import org.apache.spark.storage.BlockManagerId
+import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId}
import org.apache.spark.util.CompletionIterator
private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
- override def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics, serializer: Serializer)
+ override def fetch[T](
+ shuffleId: Int,
+ reduceId: Int,
+ context: TaskContext,
+ serializer: Serializer)
: Iterator[T] =
{
@@ -45,12 +49,12 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size))
}
- val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] = splitsByAddress.toSeq.map {
+ val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])] = splitsByAddress.toSeq.map {
case (address, splits) =>
- (address, splits.map(s => ("shuffle_%d_%d_%d".format(shuffleId, s._1, reduceId), s._2)))
+ (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2)))
}
- def unpackBlock(blockPair: (String, Option[Iterator[Any]])) : Iterator[T] = {
+ def unpackBlock(blockPair: (BlockId, Option[Iterator[Any]])) : Iterator[T] = {
val blockId = blockPair._1
val blockOption = blockPair._2
blockOption match {
@@ -58,9 +62,8 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
block.asInstanceOf[Iterator[T]]
}
case None => {
- val regex = "shuffle_([0-9]*)_([0-9]*)_([0-9]*)".r
blockId match {
- case regex(shufId, mapId, _) =>
+ case ShuffleBlockId(shufId, mapId, _) =>
val address = statuses(mapId.toInt)._1
throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, null)
case _ =>
@@ -74,7 +77,7 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer)
val itr = blockFetcherItr.flatMap(unpackBlock)
- CompletionIterator[T, Iterator[T]](itr, {
+ val completionIter = CompletionIterator[T, Iterator[T]](itr, {
val shuffleMetrics = new ShuffleReadMetrics
shuffleMetrics.shuffleFinishTime = System.currentTimeMillis
shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime
@@ -83,7 +86,9 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin
shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks
shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks
shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks
- metrics.shuffleReadMetrics = Some(shuffleMetrics)
+ context.taskMetrics.shuffleReadMetrics = Some(shuffleMetrics)
})
+
+ new InterruptibleIterator[T](context, completionIter)
}
}
diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala
index e299a106ee..519ecde50a 100644
--- a/core/src/main/scala/org/apache/spark/CacheManager.scala
+++ b/core/src/main/scala/org/apache/spark/CacheManager.scala
@@ -18,7 +18,7 @@
package org.apache.spark
import scala.collection.mutable.{ArrayBuffer, HashSet}
-import org.apache.spark.storage.{BlockManager, StorageLevel}
+import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, RDDBlockId}
import org.apache.spark.rdd.RDD
@@ -26,28 +26,29 @@ import org.apache.spark.rdd.RDD
sure a node doesn't load two copies of an RDD at once.
*/
private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
- private val loading = new HashSet[String]
+
+ /** Keys of RDD splits that are being computed/loaded. */
+ private val loading = new HashSet[RDDBlockId]()
/** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */
def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, storageLevel: StorageLevel)
: Iterator[T] = {
- val key = "rdd_%d_%d".format(rdd.id, split.index)
- logInfo("Cache key is " + key)
+ val key = RDDBlockId(rdd.id, split.index)
+ logDebug("Looking for partition " + key)
blockManager.get(key) match {
- case Some(cachedValues) =>
- // Partition is in cache, so just return its values
- logInfo("Found partition in cache!")
- return cachedValues.asInstanceOf[Iterator[T]]
+ case Some(values) =>
+ // Partition is already materialized, so just return its values
+ return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]])
case None =>
// Mark the split as loading (unless someone else marks it first)
loading.synchronized {
if (loading.contains(key)) {
- logInfo("Loading contains " + key + ", waiting...")
+ logInfo("Another thread is loading %s, waiting for it to finish...".format(key))
while (loading.contains(key)) {
try {loading.wait()} catch {case _ : Throwable =>}
}
- logInfo("Loading no longer contains " + key + ", so returning cached result")
+ logInfo("Finished waiting for %s".format(key))
// See whether someone else has successfully loaded it. The main way this would fail
// is for the RDD-level cache eviction policy if someone else has loaded the same RDD
// partition but we didn't want to make space for it. However, that case is unlikely
@@ -55,9 +56,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
// downside of the current code is that threads wait serially if this does happen.
blockManager.get(key) match {
case Some(values) =>
- return values.asInstanceOf[Iterator[T]]
+ return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]])
case None =>
- logInfo("Whoever was loading " + key + " failed; we'll try it ourselves")
+ logInfo("Whoever was loading %s failed; we'll try it ourselves".format(key))
loading.add(key)
}
} else {
@@ -66,11 +67,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
}
try {
// If we got here, we have to load the split
+ logInfo("Partition %s not found, computing it".format(key))
+ val computedValues = rdd.computeOrReadCheckpoint(split, context)
+ // Persist the result, so long as the task is not running locally
+ if (context.runningLocally) { return computedValues }
val elements = new ArrayBuffer[Any]
- logInfo("Computing partition " + split)
- elements ++= rdd.computeOrReadCheckpoint(split, context)
- // Try to put this block in the blockManager
- blockManager.put(key, elements, storageLevel, true)
+ elements ++= computedValues
+ blockManager.put(key, elements, storageLevel, tellMaster = true)
return elements.iterator.asInstanceOf[Iterator[T]]
} finally {
loading.synchronized {
diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
new file mode 100644
index 0000000000..1ad9240cfa
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -0,0 +1,250 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+import scala.concurrent._
+import scala.concurrent.duration.Duration
+import scala.util.Try
+
+import org.apache.spark.scheduler.{JobSucceeded, JobWaiter}
+import org.apache.spark.scheduler.JobFailed
+import org.apache.spark.rdd.RDD
+
+
+/**
+ * A future for the result of an action. This is an extension of the Scala Future interface to
+ * support cancellation.
+ */
+trait FutureAction[T] extends Future[T] {
+ // Note that we redefine methods of the Future trait here explicitly so we can specify a different
+ // documentation (with reference to the word "action").
+
+ /**
+ * Cancels the execution of this action.
+ */
+ def cancel()
+
+ /**
+ * Blocks until this action completes.
+ * @param atMost maximum wait time, which may be negative (no waiting is done), Duration.Inf
+ * for unbounded waiting, or a finite positive duration
+ * @return this FutureAction
+ */
+ override def ready(atMost: Duration)(implicit permit: CanAwait): FutureAction.this.type
+
+ /**
+ * Awaits and returns the result (of type T) of this action.
+ * @param atMost maximum wait time, which may be negative (no waiting is done), Duration.Inf
+ * for unbounded waiting, or a finite positive duration
+ * @throws Exception exception during action execution
+ * @return the result value if the action is completed within the specific maximum wait time
+ */
+ @throws(classOf[Exception])
+ override def result(atMost: Duration)(implicit permit: CanAwait): T
+
+ /**
+ * When this action is completed, either through an exception, or a value, applies the provided
+ * function.
+ */
+ def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext)
+
+ /**
+ * Returns whether the action has already been completed with a value or an exception.
+ */
+ override def isCompleted: Boolean
+
+ /**
+ * The value of this Future.
+ *
+ * If the future is not completed the returned value will be None. If the future is completed
+ * the value will be Some(Success(t)) if it contains a valid result, or Some(Failure(error)) if
+ * it contains an exception.
+ */
+ override def value: Option[Try[T]]
+
+ /**
+ * Blocks and returns the result of this job.
+ */
+ @throws(classOf[Exception])
+ def get(): T = Await.result(this, Duration.Inf)
+}
+
+
+/**
+ * The future holding the result of an action that triggers a single job. Examples include
+ * count, collect, reduce.
+ */
+class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T)
+ extends FutureAction[T] {
+
+ override def cancel() {
+ jobWaiter.cancel()
+ }
+
+ override def ready(atMost: Duration)(implicit permit: CanAwait): SimpleFutureAction.this.type = {
+ if (!atMost.isFinite()) {
+ awaitResult()
+ } else {
+ val finishTime = System.currentTimeMillis() + atMost.toMillis
+ while (!isCompleted) {
+ val time = System.currentTimeMillis()
+ if (time >= finishTime) {
+ throw new TimeoutException
+ } else {
+ jobWaiter.wait(finishTime - time)
+ }
+ }
+ }
+ this
+ }
+
+ @throws(classOf[Exception])
+ override def result(atMost: Duration)(implicit permit: CanAwait): T = {
+ ready(atMost)(permit)
+ awaitResult() match {
+ case scala.util.Success(res) => res
+ case scala.util.Failure(e) => throw e
+ }
+ }
+
+ override def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext) {
+ executor.execute(new Runnable {
+ override def run() {
+ func(awaitResult())
+ }
+ })
+ }
+
+ override def isCompleted: Boolean = jobWaiter.jobFinished
+
+ override def value: Option[Try[T]] = {
+ if (jobWaiter.jobFinished) {
+ Some(awaitResult())
+ } else {
+ None
+ }
+ }
+
+ private def awaitResult(): Try[T] = {
+ jobWaiter.awaitResult() match {
+ case JobSucceeded => scala.util.Success(resultFunc)
+ case JobFailed(e: Exception, _) => scala.util.Failure(e)
+ }
+ }
+}
+
+
+/**
+ * A FutureAction for actions that could trigger multiple Spark jobs. Examples include take,
+ * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the
+ * action thread if it is being blocked by a job.
+ */
+class ComplexFutureAction[T] extends FutureAction[T] {
+
+ // Pointer to the thread that is executing the action. It is set when the action is run.
+ @volatile private var thread: Thread = _
+
+ // A flag indicating whether the future has been cancelled. This is used in case the future
+ // is cancelled before the action was even run (and thus we have no thread to interrupt).
+ @volatile private var _cancelled: Boolean = false
+
+ // A promise used to signal the future.
+ private val p = promise[T]()
+
+ override def cancel(): Unit = this.synchronized {
+ _cancelled = true
+ if (thread != null) {
+ thread.interrupt()
+ }
+ }
+
+ /**
+ * Executes some action enclosed in the closure. To properly enable cancellation, the closure
+ * should use runJob implementation in this promise. See takeAsync for example.
+ */
+ def run(func: => T)(implicit executor: ExecutionContext): this.type = {
+ scala.concurrent.future {
+ thread = Thread.currentThread
+ try {
+ p.success(func)
+ } catch {
+ case e: Exception => p.failure(e)
+ } finally {
+ thread = null
+ }
+ }
+ this
+ }
+
+ /**
+ * Runs a Spark job. This is a wrapper around the same functionality provided by SparkContext
+ * to enable cancellation.
+ */
+ def runJob[T, U, R](
+ rdd: RDD[T],
+ processPartition: Iterator[T] => U,
+ partitions: Seq[Int],
+ resultHandler: (Int, U) => Unit,
+ resultFunc: => R) {
+ // If the action hasn't been cancelled yet, submit the job. The check and the submitJob
+ // command need to be in an atomic block.
+ val job = this.synchronized {
+ if (!cancelled) {
+ rdd.context.submitJob(rdd, processPartition, partitions, resultHandler, resultFunc)
+ } else {
+ throw new SparkException("Action has been cancelled")
+ }
+ }
+
+ // Wait for the job to complete. If the action is cancelled (with an interrupt),
+ // cancel the job and stop the execution. This is not in a synchronized block because
+ // Await.ready eventually waits on the monitor in FutureJob.jobWaiter.
+ try {
+ Await.ready(job, Duration.Inf)
+ } catch {
+ case e: InterruptedException =>
+ job.cancel()
+ throw new SparkException("Action has been cancelled")
+ }
+ }
+
+ /**
+ * Returns whether the promise has been cancelled.
+ */
+ def cancelled: Boolean = _cancelled
+
+ @throws(classOf[InterruptedException])
+ @throws(classOf[scala.concurrent.TimeoutException])
+ override def ready(atMost: Duration)(implicit permit: CanAwait): this.type = {
+ p.future.ready(atMost)(permit)
+ this
+ }
+
+ @throws(classOf[Exception])
+ override def result(atMost: Duration)(implicit permit: CanAwait): T = {
+ p.future.result(atMost)(permit)
+ }
+
+ override def onComplete[U](func: (Try[T]) => U)(implicit executor: ExecutionContext): Unit = {
+ p.future.onComplete(func)(executor)
+ }
+
+ override def isCompleted: Boolean = p.isCompleted
+
+ override def value: Option[Try[T]] = p.future.value
+}
diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala
new file mode 100644
index 0000000000..56e0b8d2c0
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala
@@ -0,0 +1,30 @@
+/*
+ * 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
+
+/**
+ * An iterator that wraps around an existing iterator to provide task killing functionality.
+ * It works by checking the interrupted flag in TaskContext.
+ */
+class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T])
+ extends Iterator[T] {
+
+ def hasNext: Boolean = !context.interrupted && delegate.hasNext
+
+ def next(): T = delegate.next()
+}
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index ae7cf2a893..1e3f1ebfaf 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -32,7 +32,7 @@ import akka.util.Duration
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage.BlockManagerId
-import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.util.{MetadataCleanerType, Utils, MetadataCleaner, TimeStampedHashMap}
private[spark] sealed trait MapOutputTrackerMessage
@@ -71,7 +71,7 @@ private[spark] class MapOutputTracker extends Logging {
var cacheEpoch = epoch
private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]
- val metadataCleaner = new MetadataCleaner("MapOutputTracker", this.cleanup)
+ val metadataCleaner = new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup)
// Send a message to the trackerActor and get its result within a default timeout, or
// throw a SparkException if this fails.
diff --git a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
index 307c383a89..a85aa50a9b 100644
--- a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
+++ b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala
@@ -27,7 +27,10 @@ private[spark] abstract class ShuffleFetcher {
* Fetch the shuffle outputs for a given ShuffleDependency.
* @return An iterator over the elements of the fetched shuffle outputs.
*/
- def fetch[T](shuffleId: Int, reduceId: Int, metrics: TaskMetrics,
+ def fetch[T](
+ shuffleId: Int,
+ reduceId: Int,
+ context: TaskContext,
serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[T]
/** Stop the fetcher */
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index f2641851cb..f674cb397f 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -27,7 +27,6 @@ import scala.collection.generic.Growable
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
-import scala.util.DynamicVariable
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
@@ -52,17 +51,22 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFor
import org.apache.mesos.MesosNativeLibrary
+import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.LocalSparkCluster
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd._
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.{StandaloneSchedulerBackend, SparkDeploySchedulerBackend,
- ClusterScheduler, Schedulable, SchedulingMode}
+ ClusterScheduler}
import org.apache.spark.scheduler.local.LocalScheduler
-import org.apache.spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
+import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
import org.apache.spark.storage.{StorageUtils, BlockManagerSource}
import org.apache.spark.ui.SparkUI
-import org.apache.spark.util.{ClosureCleaner, Utils, MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.util._
+import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.storage.RDDInfo
+import org.apache.spark.storage.StorageStatus
+import scala.Some
import org.apache.spark.scheduler.StageInfo
import org.apache.spark.storage.RDDInfo
import org.apache.spark.storage.StorageStatus
@@ -84,9 +88,11 @@ class SparkContext(
val sparkHome: String = null,
val jars: Seq[String] = Nil,
val environment: Map[String, String] = Map(),
- // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc) too.
- // This is typically generated from InputFormatInfo.computePreferredLocations .. host, set of data-local splits on host
- val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = scala.collection.immutable.Map())
+ // This is used only by yarn for now, but should be relevant to other cluster types (mesos, etc)
+ // too. This is typically generated from InputFormatInfo.computePreferredLocations .. host, set
+ // of data-local splits on host
+ val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
+ scala.collection.immutable.Map())
extends Logging {
// Ensure logging is initialized before we spawn any threads
@@ -117,7 +123,7 @@ class SparkContext(
// Keeps track of all persisted RDDs
private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
- private[spark] val metadataCleaner = new MetadataCleaner("SparkContext", this.cleanup)
+ private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup)
// Initalize the Spark UI
private[spark] val ui = new SparkUI(this)
@@ -146,7 +152,7 @@ class SparkContext(
}
// Create and start the scheduler
- private var taskScheduler: TaskScheduler = {
+ private[spark] var taskScheduler: TaskScheduler = {
// Regular expression used for local[N] master format
val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r
// Regular expression for local[N, maxRetries], used in tests with failing tasks
@@ -154,7 +160,7 @@ class SparkContext(
// Regular expression for simulating a Spark cluster of [N, cores, memory] locally
val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r
// Regular expression for connecting to Spark deploy clusters
- val SPARK_REGEX = """(spark://.*)""".r
+ val SPARK_REGEX = """spark://(.*)""".r
//Regular expression for connection to Mesos cluster
val MESOS_REGEX = """(mesos://.*)""".r
@@ -170,7 +176,8 @@ class SparkContext(
case SPARK_REGEX(sparkUrl) =>
val scheduler = new ClusterScheduler(this)
- val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
+ val masterUrls = sparkUrl.split(",").map("spark://" + _)
+ val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName)
scheduler.initialize(backend)
scheduler
@@ -186,8 +193,8 @@ class SparkContext(
val scheduler = new ClusterScheduler(this)
val localCluster = new LocalSparkCluster(
numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
- val sparkUrl = localCluster.start()
- val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
+ val masterUrls = localCluster.start()
+ val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName)
scheduler.initialize(backend)
backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
localCluster.stop()
@@ -196,7 +203,7 @@ class SparkContext(
case "yarn-standalone" =>
val scheduler = try {
- val clazz = Class.forName("spark.scheduler.cluster.YarnClusterScheduler")
+ val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
val cons = clazz.getConstructor(classOf[SparkContext])
cons.newInstance(this).asInstanceOf[ClusterScheduler]
} catch {
@@ -229,7 +236,7 @@ class SparkContext(
}
taskScheduler.start()
- @volatile private var dagScheduler = new DAGScheduler(taskScheduler)
+ @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler)
dagScheduler.start()
ui.start()
@@ -239,7 +246,8 @@ class SparkContext(
val env = SparkEnv.get
val conf = env.hadoop.newConfiguration()
// Explicitly check for S3 environment variables
- if (System.getenv("AWS_ACCESS_KEY_ID") != null && System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
+ if (System.getenv("AWS_ACCESS_KEY_ID") != null &&
+ System.getenv("AWS_SECRET_ACCESS_KEY") != null) {
conf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
conf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
conf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
@@ -257,23 +265,28 @@ class SparkContext(
private[spark] var checkpointDir: Option[String] = None
// Thread Local variable that can be used by users to pass information down the stack
- private val localProperties = new DynamicVariable[Properties](null)
+ private val localProperties = new InheritableThreadLocal[Properties] {
+ override protected def childValue(parent: Properties): Properties = new Properties(parent)
+ }
def initLocalProperties() {
- localProperties.value = new Properties()
+ localProperties.set(new Properties())
}
def setLocalProperty(key: String, value: String) {
- if (localProperties.value == null) {
- localProperties.value = new Properties()
+ if (localProperties.get() == null) {
+ localProperties.set(new Properties())
}
if (value == null) {
- localProperties.value.remove(key)
+ localProperties.get.remove(key)
} else {
- localProperties.value.setProperty(key, value)
+ localProperties.get.setProperty(key, value)
}
}
+ def getLocalProperty(key: String): String =
+ Option(localProperties.get).map(_.getProperty(key)).getOrElse(null)
+
/** Set a human readable description of the current job. */
def setJobDescription(value: String) {
setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value)
@@ -282,8 +295,8 @@ class SparkContext(
// Post init
taskScheduler.postStartHook()
- val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler)
- val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager)
+ val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this)
+ val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this)
def initDriverMetrics() {
SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource)
@@ -322,7 +335,7 @@ class SparkContext(
}
/**
- * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any
+ * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and any
* other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
* etc).
*/
@@ -333,6 +346,8 @@ class SparkContext(
valueClass: Class[V],
minSplits: Int = defaultMinSplits
): RDD[(K, V)] = {
+ // Add necessary security credentials to the JobConf before broadcasting it.
+ SparkEnv.get.hadoop.addCredentials(conf)
new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits)
}
@@ -343,10 +358,18 @@ class SparkContext(
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int = defaultMinSplits
- ) : RDD[(K, V)] = {
- val conf = new JobConf(hadoopConfiguration)
- FileInputFormat.setInputPaths(conf, path)
- new HadoopRDD(this, conf, inputFormatClass, keyClass, valueClass, minSplits)
+ ): RDD[(K, V)] = {
+ // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it.
+ val confBroadcast = broadcast(new SerializableWritable(hadoopConfiguration))
+ val setInputPathsFunc = (jobConf: JobConf) => FileInputFormat.setInputPaths(jobConf, path)
+ new HadoopRDD(
+ this,
+ confBroadcast,
+ Some(setInputPathsFunc),
+ inputFormatClass,
+ keyClass,
+ valueClass,
+ minSplits)
}
/**
@@ -644,10 +667,21 @@ class SparkContext(
key = uri.getScheme match {
case null | "file" =>
if (env.hadoop.isYarnMode()) {
- logWarning("local jar specified as parameter to addJar under Yarn mode")
- return
+ // In order for this to work on yarn the user must specify the --addjars option to
+ // the client to upload the file into the distributed cache to make it show up in the
+ // current working directory.
+ val fileName = new Path(uri.getPath).getName()
+ try {
+ env.httpFileServer.addJar(new File(fileName))
+ } catch {
+ case e: Exception => {
+ logError("Error adding jar (" + e + "), was the --addJars option used?")
+ throw e
+ }
+ }
+ } else {
+ env.httpFileServer.addJar(new File(uri.getPath))
}
- env.httpFileServer.addJar(new File(uri.getPath))
case _ =>
path
}
@@ -721,9 +755,11 @@ class SparkContext(
allowLocal: Boolean,
resultHandler: (Int, U) => Unit) {
val callSite = Utils.formatSparkCallSite
+ val cleanedFunc = clean(func)
logInfo("Starting job: " + callSite)
val start = System.nanoTime
- val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal, resultHandler, localProperties.value)
+ val result = dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
+ resultHandler, localProperties.get)
logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
rdd.doCheckpoint()
result
@@ -806,12 +842,43 @@ class SparkContext(
val callSite = Utils.formatSparkCallSite
logInfo("Starting job: " + callSite)
val start = System.nanoTime
- val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.value)
+ val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout,
+ localProperties.get)
logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
result
}
/**
+ * Submit a job for execution and return a FutureJob holding the result.
+ */
+ def submitJob[T, U, R](
+ rdd: RDD[T],
+ processPartition: Iterator[T] => U,
+ partitions: Seq[Int],
+ resultHandler: (Int, U) => Unit,
+ resultFunc: => R): SimpleFutureAction[R] =
+ {
+ val cleanF = clean(processPartition)
+ val callSite = Utils.formatSparkCallSite
+ val waiter = dagScheduler.submitJob(
+ rdd,
+ (context: TaskContext, iter: Iterator[T]) => cleanF(iter),
+ partitions,
+ callSite,
+ allowLocal = false,
+ resultHandler,
+ null)
+ new SimpleFutureAction(waiter, resultFunc)
+ }
+
+ /**
+ * Cancel all jobs that have been scheduled or are running.
+ */
+ def cancelAllJobs() {
+ dagScheduler.cancelAllJobs()
+ }
+
+ /**
* Clean a closure to make it ready to serialized and send to tasks
* (removes unreferenced variables in $outer's, updates REPL variables)
*/
@@ -894,6 +961,8 @@ object SparkContext {
implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) =
new PairRDDFunctions(rdd)
+ implicit def rddToAsyncRDDActions[T: ClassManifest](rdd: RDD[T]) = new AsyncRDDActions(rdd)
+
implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest](
rdd: RDD[(K, V)]) =
new SequenceFileRDDFunctions(rdd)
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index 478e5a0aaf..29968c273c 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -62,7 +62,7 @@ class SparkEnv (
val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
if(yarnMode) {
try {
- Class.forName("spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil]
+ Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil").newInstance.asInstanceOf[SparkHadoopUtil]
} catch {
case th: Throwable => throw new SparkException("Unable to load YARN support", th)
}
diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index b2dd668330..51584d686d 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -17,20 +17,30 @@
package org.apache.spark
-import executor.TaskMetrics
import scala.collection.mutable.ArrayBuffer
+import org.apache.spark.executor.TaskMetrics
+
class TaskContext(
- val stageId: Int,
- val splitId: Int,
+ private[spark] val stageId: Int,
+ val partitionId: Int,
val attemptId: Long,
- val taskMetrics: TaskMetrics = TaskMetrics.empty()
+ val runningLocally: Boolean = false,
+ @volatile var interrupted: Boolean = false,
+ private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty()
) extends Serializable {
- @transient val onCompleteCallbacks = new ArrayBuffer[() => Unit]
+ @deprecated("use partitionId", "0.8.1")
+ def splitId = partitionId
+
+ // List of callback functions to execute when the task completes.
+ @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit]
- // Add a callback function to be executed on task completion. An example use
- // is for HadoopRDD to register a callback to close the input stream.
+ /**
+ * Add a callback function to be executed on task completion. An example use
+ * is for HadoopRDD to register a callback to close the input stream.
+ * @param f Callback function.
+ */
def addOnCompleteCallback(f: () => Unit) {
onCompleteCallbacks += f
}
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index 03bf268863..c1e5e04b31 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -46,6 +46,12 @@ private[spark] case class ExceptionFailure(
metrics: Option[TaskMetrics])
extends TaskEndReason
-private[spark] case class OtherFailure(message: String) extends TaskEndReason
+/**
+ * The task finished successfully, but the result was lost from the executor's block manager before
+ * it was fetched.
+ */
+private[spark] case object TaskResultLost extends TaskEndReason
-private[spark] case class TaskResultTooBigFailure() extends TaskEndReason
+private[spark] case object TaskKilled extends TaskEndReason
+
+private[spark] case class OtherFailure(message: String) extends TaskEndReason
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 7e6e691f11..7a3568c5ef 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
@@ -68,6 +68,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
new JavaRDD(rdd.map(f)(f.returnType()))(f.returnType())
/**
+ * 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](
+ 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))),
+ preservesPartitioning))
+
+ /**
* Return a new RDD by applying a function to all elements of this RDD.
*/
def map[R](f: DoubleFunction[T]): JavaDoubleRDD =
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala
index b090c6edf3..2be4e323be 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonPartitioner.scala
@@ -17,12 +17,13 @@
package org.apache.spark.api.python
-import org.apache.spark.Partitioner
import java.util.Arrays
+
+import org.apache.spark.Partitioner
import org.apache.spark.util.Utils
/**
- * A [[org.apache.spark.Partitioner]] that performs handling of byte arrays, for use by the Python API.
+ * A [[org.apache.spark.Partitioner]] that performs handling of long-valued keys, for use by the Python API.
*
* Stores the unique id() of the Python-side partitioning function so that it is incorporated into
* equality comparisons. Correctness requires that the id is a unique identifier for the
@@ -30,6 +31,7 @@ import org.apache.spark.util.Utils
* function). This can be ensured by using the Python id() function and maintaining a reference
* to the Python partitioning function so that its id() is not reused.
*/
+
private[spark] class PythonPartitioner(
override val numPartitions: Int,
val pyPartitionFunctionId: Long)
@@ -37,7 +39,9 @@ private[spark] class PythonPartitioner(
override def getPartition(key: Any): Int = key match {
case null => 0
- case key: Array[Byte] => Utils.nonNegativeMod(Arrays.hashCode(key), numPartitions)
+ // we don't trust the Python partition function to return valid partition ID's so
+ // let's do a modulo numPartitions in any case
+ case key: Long => Utils.nonNegativeMod(key.toInt, numPartitions)
case _ => Utils.nonNegativeMod(key.hashCode(), numPartitions)
}
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 ccd3833964..1f8ad688a6 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
@@ -187,14 +187,14 @@ private class PythonException(msg: String) extends Exception(msg)
* This is used by PySpark's shuffle operations.
*/
private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
- RDD[(Array[Byte], Array[Byte])](prev) {
+ RDD[(Long, Array[Byte])](prev) {
override def getPartitions = prev.partitions
override def compute(split: Partition, context: TaskContext) =
prev.iterator(split, context).grouped(2).map {
- case Seq(a, b) => (a, b)
+ case Seq(a, b) => (Utils.deserializeLongValue(a), b)
case x => throw new SparkException("PairwiseRDD: unexpected value: " + x)
}
- val asJavaPairRDD : JavaPairRDD[Array[Byte], Array[Byte]] = JavaPairRDD.fromRDD(this)
+ val asJavaPairRDD : JavaPairRDD[Long, Array[Byte]] = JavaPairRDD.fromRDD(this)
}
private[spark] object PythonRDD {
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
index 93e7815ab5..b6c484bfe1 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BitTorrentBroadcast.scala
@@ -26,7 +26,7 @@ import scala.collection.mutable.{ListBuffer, Map, Set}
import scala.math
import org.apache.spark._
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
import org.apache.spark.util.Utils
private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
@@ -36,7 +36,7 @@ private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal:
def value = value_
- def blockId: String = "broadcast_" + id
+ def blockId = BroadcastBlockId(id)
MultiTracker.synchronized {
SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
index 9db26ae6de..609464e38d 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -25,16 +25,15 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
import org.apache.spark.{HttpServer, Logging, SparkEnv}
import org.apache.spark.io.CompressionCodec
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.{Utils, MetadataCleaner, TimeStampedHashSet}
-
+import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
+import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils}
private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
extends Broadcast[T](id) with Logging with Serializable {
def value = value_
- def blockId: String = "broadcast_" + id
+ def blockId = BroadcastBlockId(id)
HttpBroadcast.synchronized {
SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
@@ -82,7 +81,7 @@ private object HttpBroadcast extends Logging {
private var server: HttpServer = null
private val files = new TimeStampedHashSet[String]
- private val cleaner = new MetadataCleaner("HttpBroadcast", cleanup)
+ private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup)
private lazy val compressionCodec = CompressionCodec.createCodec()
@@ -121,7 +120,7 @@ private object HttpBroadcast extends Logging {
}
def write(id: Long, value: Any) {
- val file = new File(broadcastDir, "broadcast-" + id)
+ val file = new File(broadcastDir, BroadcastBlockId(id).name)
val out: OutputStream = {
if (compress) {
compressionCodec.compressedOutputStream(new FileOutputStream(file))
@@ -137,7 +136,7 @@ private object HttpBroadcast extends Logging {
}
def read[T](id: Long): T = {
- val url = serverUri + "/broadcast-" + id
+ val url = serverUri + "/" + BroadcastBlockId(id).name
val in = {
if (compress) {
compressionCodec.compressedInputStream(new URL(url).openStream())
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
index 80c97ca073..e6674d49a7 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TreeBroadcast.scala
@@ -19,13 +19,11 @@ package org.apache.spark.broadcast
import java.io._
import java.net._
-import java.util.{Comparator, Random, UUID}
-import scala.collection.mutable.{ListBuffer, Map, Set}
-import scala.math
+import scala.collection.mutable.{ListBuffer, Set}
import org.apache.spark._
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{BroadcastBlockId, StorageLevel}
import org.apache.spark.util.Utils
private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
@@ -33,7 +31,7 @@ extends Broadcast[T](id) with Logging with Serializable {
def value = value_
- def blockId = "broadcast_" + id
+ def blockId = BroadcastBlockId(id)
MultiTracker.synchronized {
SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
index c31619db27..275331724a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
@@ -21,12 +21,14 @@ import scala.collection.immutable.List
import org.apache.spark.deploy.ExecutorState.ExecutorState
import org.apache.spark.deploy.master.{WorkerInfo, ApplicationInfo}
+import org.apache.spark.deploy.master.RecoveryState.MasterState
import org.apache.spark.deploy.worker.ExecutorRunner
import org.apache.spark.util.Utils
private[deploy] sealed trait DeployMessage extends Serializable
+/** Contains messages sent between Scheduler actor nodes. */
private[deploy] object DeployMessages {
// Worker to Master
@@ -52,17 +54,20 @@ private[deploy] object DeployMessages {
exitStatus: Option[Int])
extends DeployMessage
+ case class WorkerSchedulerStateResponse(id: String, executors: List[ExecutorDescription])
+
case class Heartbeat(workerId: String) extends DeployMessage
// Master to Worker
- case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
+ case class RegisteredWorker(masterUrl: String, masterWebUiUrl: String) extends DeployMessage
case class RegisterWorkerFailed(message: String) extends DeployMessage
- case class KillExecutor(appId: String, execId: Int) extends DeployMessage
+ case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage
case class LaunchExecutor(
+ masterUrl: String,
appId: String,
execId: Int,
appDesc: ApplicationDescription,
@@ -76,9 +81,11 @@ private[deploy] object DeployMessages {
case class RegisterApplication(appDescription: ApplicationDescription)
extends DeployMessage
+ case class MasterChangeAcknowledged(appId: String)
+
// Master to Client
- case class RegisteredApplication(appId: String) extends DeployMessage
+ case class RegisteredApplication(appId: String, masterUrl: String) extends DeployMessage
// TODO(matei): replace hostPort with host
case class ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) {
@@ -94,6 +101,10 @@ private[deploy] object DeployMessages {
case object StopClient
+ // Master to Worker & Client
+
+ case class MasterChanged(masterUrl: String, masterWebUiUrl: String)
+
// MasterWebUI To Master
case object RequestMasterState
@@ -101,7 +112,8 @@ private[deploy] object DeployMessages {
// Master to MasterWebUI
case class MasterStateResponse(host: String, port: Int, workers: Array[WorkerInfo],
- activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
+ activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo],
+ status: MasterState) {
Utils.checkHost(host, "Required hostname")
assert (port > 0)
@@ -123,8 +135,7 @@ private[deploy] object DeployMessages {
assert (port > 0)
}
- // Actor System to Master
-
- case object CheckForWorkerTimeOut
+ // Actor System to Worker
+ case object SendHeartbeat
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala
new file mode 100644
index 0000000000..2abf0b69dd
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorDescription.scala
@@ -0,0 +1,34 @@
+/*
+ * 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
+
+/**
+ * Used to send state on-the-wire about Executors from Worker to Master.
+ * This state is sufficient for the Master to reconstruct its internal data structures during
+ * failover.
+ */
+private[spark] class ExecutorDescription(
+ val appId: String,
+ val execId: Int,
+ val cores: Int,
+ val state: ExecutorState.Value)
+ extends Serializable {
+
+ override def toString: String =
+ "ExecutorState(appId=%s, execId=%d, cores=%d, state=%s)".format(appId, execId, cores, state)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
new file mode 100644
index 0000000000..668032a3a2
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -0,0 +1,420 @@
+/*
+ *
+ * * 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
+
+import java.io._
+import java.net.URL
+import java.util.concurrent.TimeoutException
+
+import scala.concurrent.{Await, future, promise}
+import scala.concurrent.duration._
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.collection.mutable.ListBuffer
+import scala.sys.process._
+
+import net.liftweb.json.JsonParser
+
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.deploy.master.RecoveryState
+
+/**
+ * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master.
+ * In order to mimic a real distributed cluster more closely, Docker is used.
+ * Execute using
+ * ./spark-class org.apache.spark.deploy.FaultToleranceTest
+ *
+ * Make sure that that the environment includes the following properties in SPARK_DAEMON_JAVA_OPTS:
+ * - spark.deploy.recoveryMode=ZOOKEEPER
+ * - spark.deploy.zookeeper.url=172.17.42.1:2181
+ * Note that 172.17.42.1 is the default docker ip for the host and 2181 is the default ZK port.
+ *
+ * Unfortunately, due to the Docker dependency this suite cannot be run automatically without a
+ * working installation of Docker. In addition to having Docker, the following are assumed:
+ * - Docker can run without sudo (see http://docs.docker.io/en/latest/use/basics/)
+ * - The docker images tagged spark-test-master and spark-test-worker are built from the
+ * docker/ directory. Run 'docker/spark-test/build' to generate these.
+ */
+private[spark] object FaultToleranceTest extends App with Logging {
+ val masters = ListBuffer[TestMasterInfo]()
+ val workers = ListBuffer[TestWorkerInfo]()
+ var sc: SparkContext = _
+
+ var numPassed = 0
+ var numFailed = 0
+
+ val sparkHome = System.getenv("SPARK_HOME")
+ assertTrue(sparkHome != null, "Run with a valid SPARK_HOME")
+
+ val containerSparkHome = "/opt/spark"
+ val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome)
+
+ System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip
+
+ def afterEach() {
+ if (sc != null) {
+ sc.stop()
+ sc = null
+ }
+ terminateCluster()
+ }
+
+ test("sanity-basic") {
+ addMasters(1)
+ addWorkers(1)
+ createClient()
+ assertValidClusterState()
+ }
+
+ test("sanity-many-masters") {
+ addMasters(3)
+ addWorkers(3)
+ createClient()
+ assertValidClusterState()
+ }
+
+ test("single-master-halt") {
+ addMasters(3)
+ addWorkers(2)
+ createClient()
+ assertValidClusterState()
+
+ killLeader()
+ delay(30 seconds)
+ assertValidClusterState()
+ createClient()
+ assertValidClusterState()
+ }
+
+ test("single-master-restart") {
+ addMasters(1)
+ addWorkers(2)
+ createClient()
+ assertValidClusterState()
+
+ killLeader()
+ addMasters(1)
+ delay(30 seconds)
+ assertValidClusterState()
+
+ killLeader()
+ addMasters(1)
+ delay(30 seconds)
+ assertValidClusterState()
+ }
+
+ test("cluster-failure") {
+ addMasters(2)
+ addWorkers(2)
+ createClient()
+ assertValidClusterState()
+
+ terminateCluster()
+ addMasters(2)
+ addWorkers(2)
+ assertValidClusterState()
+ }
+
+ test("all-but-standby-failure") {
+ addMasters(2)
+ addWorkers(2)
+ createClient()
+ assertValidClusterState()
+
+ killLeader()
+ workers.foreach(_.kill())
+ workers.clear()
+ delay(30 seconds)
+ addWorkers(2)
+ assertValidClusterState()
+ }
+
+ test("rolling-outage") {
+ addMasters(1)
+ delay()
+ addMasters(1)
+ delay()
+ addMasters(1)
+ addWorkers(2)
+ createClient()
+ assertValidClusterState()
+ assertTrue(getLeader == masters.head)
+
+ (1 to 3).foreach { _ =>
+ killLeader()
+ delay(30 seconds)
+ assertValidClusterState()
+ assertTrue(getLeader == masters.head)
+ addMasters(1)
+ }
+ }
+
+ def test(name: String)(fn: => Unit) {
+ try {
+ fn
+ numPassed += 1
+ logInfo("Passed: " + name)
+ } catch {
+ case e: Exception =>
+ numFailed += 1
+ logError("FAILED: " + name, e)
+ }
+ afterEach()
+ }
+
+ def addMasters(num: Int) {
+ (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) }
+ }
+
+ def addWorkers(num: Int) {
+ val masterUrls = getMasterUrls(masters)
+ (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) }
+ }
+
+ /** Creates a SparkContext, which constructs a Client to interact with our cluster. */
+ def createClient() = {
+ if (sc != null) { sc.stop() }
+ // Counter-hack: Because of a hack in SparkEnv#createFromSystemProperties() that changes this
+ // property, we need to reset it.
+ System.setProperty("spark.driver.port", "0")
+ sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome)
+ }
+
+ def getMasterUrls(masters: Seq[TestMasterInfo]): String = {
+ "spark://" + masters.map(master => master.ip + ":7077").mkString(",")
+ }
+
+ def getLeader: TestMasterInfo = {
+ val leaders = masters.filter(_.state == RecoveryState.ALIVE)
+ assertTrue(leaders.size == 1)
+ leaders(0)
+ }
+
+ def killLeader(): Unit = {
+ masters.foreach(_.readState())
+ val leader = getLeader
+ masters -= leader
+ leader.kill()
+ }
+
+ def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis)
+
+ def terminateCluster() {
+ masters.foreach(_.kill())
+ workers.foreach(_.kill())
+ masters.clear()
+ workers.clear()
+ }
+
+ /** This includes Client retry logic, so it may take a while if the cluster is recovering. */
+ def assertUsable() = {
+ val f = future {
+ try {
+ val res = sc.parallelize(0 until 10).collect()
+ assertTrue(res.toList == (0 until 10))
+ true
+ } catch {
+ case e: Exception =>
+ logError("assertUsable() had exception", e)
+ e.printStackTrace()
+ false
+ }
+ }
+
+ // Avoid waiting indefinitely (e.g., we could register but get no executors).
+ assertTrue(Await.result(f, 120 seconds))
+ }
+
+ /**
+ * Asserts that the cluster is usable and that the expected masters and workers
+ * are all alive in a proper configuration (e.g., only one leader).
+ */
+ def assertValidClusterState() = {
+ assertUsable()
+ var numAlive = 0
+ var numStandby = 0
+ var numLiveApps = 0
+ var liveWorkerIPs: Seq[String] = List()
+
+ def stateValid(): Boolean = {
+ (workers.map(_.ip) -- liveWorkerIPs).isEmpty &&
+ numAlive == 1 && numStandby == masters.size - 1 && numLiveApps >= 1
+ }
+
+ val f = future {
+ try {
+ while (!stateValid()) {
+ Thread.sleep(1000)
+
+ numAlive = 0
+ numStandby = 0
+ numLiveApps = 0
+
+ masters.foreach(_.readState())
+
+ for (master <- masters) {
+ master.state match {
+ case RecoveryState.ALIVE =>
+ numAlive += 1
+ liveWorkerIPs = master.liveWorkerIPs
+ case RecoveryState.STANDBY =>
+ numStandby += 1
+ case _ => // ignore
+ }
+
+ numLiveApps += master.numLiveApps
+ }
+ }
+ true
+ } catch {
+ case e: Exception =>
+ logError("assertValidClusterState() had exception", e)
+ false
+ }
+ }
+
+ try {
+ assertTrue(Await.result(f, 120 seconds))
+ } catch {
+ case e: TimeoutException =>
+ logError("Master states: " + masters.map(_.state))
+ logError("Num apps: " + numLiveApps)
+ logError("IPs expected: " + workers.map(_.ip) + " / found: " + liveWorkerIPs)
+ throw new RuntimeException("Failed to get into acceptable cluster state after 2 min.", e)
+ }
+ }
+
+ def assertTrue(bool: Boolean, message: String = "") {
+ if (!bool) {
+ throw new IllegalStateException("Assertion failed: " + message)
+ }
+ }
+
+ logInfo("Ran %s tests, %s passed and %s failed".format(numPassed+numFailed, numPassed, numFailed))
+}
+
+private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File)
+ extends Logging {
+
+ implicit val formats = net.liftweb.json.DefaultFormats
+ var state: RecoveryState.Value = _
+ var liveWorkerIPs: List[String] = _
+ var numLiveApps = 0
+
+ logDebug("Created master: " + this)
+
+ def readState() {
+ try {
+ val masterStream = new InputStreamReader(new URL("http://%s:8080/json".format(ip)).openStream)
+ val json = JsonParser.parse(masterStream, closeAutomatically = true)
+
+ val workers = json \ "workers"
+ val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE")
+ liveWorkerIPs = liveWorkers.map(w => (w \ "host").extract[String])
+
+ numLiveApps = (json \ "activeapps").children.size
+
+ val status = json \\ "status"
+ val stateString = status.extract[String]
+ state = RecoveryState.values.filter(state => state.toString == stateString).head
+ } catch {
+ case e: Exception =>
+ // ignore, no state update
+ logWarning("Exception", e)
+ }
+ }
+
+ def kill() { Docker.kill(dockerId) }
+
+ override def toString: String =
+ "[ip=%s, id=%s, logFile=%s, state=%s]".
+ format(ip, dockerId.id, logFile.getAbsolutePath, state)
+}
+
+private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File)
+ extends Logging {
+
+ implicit val formats = net.liftweb.json.DefaultFormats
+
+ logDebug("Created worker: " + this)
+
+ def kill() { Docker.kill(dockerId) }
+
+ override def toString: String =
+ "[ip=%s, id=%s, logFile=%s]".format(ip, dockerId, logFile.getAbsolutePath)
+}
+
+private[spark] object SparkDocker {
+ def startMaster(mountDir: String): TestMasterInfo = {
+ val cmd = Docker.makeRunCmd("spark-test-master", mountDir = mountDir)
+ val (ip, id, outFile) = startNode(cmd)
+ new TestMasterInfo(ip, id, outFile)
+ }
+
+ def startWorker(mountDir: String, masters: String): TestWorkerInfo = {
+ val cmd = Docker.makeRunCmd("spark-test-worker", args = masters, mountDir = mountDir)
+ val (ip, id, outFile) = startNode(cmd)
+ new TestWorkerInfo(ip, id, outFile)
+ }
+
+ private def startNode(dockerCmd: ProcessBuilder) : (String, DockerId, File) = {
+ val ipPromise = promise[String]()
+ val outFile = File.createTempFile("fault-tolerance-test", "")
+ outFile.deleteOnExit()
+ val outStream: FileWriter = new FileWriter(outFile)
+ def findIpAndLog(line: String): Unit = {
+ if (line.startsWith("CONTAINER_IP=")) {
+ val ip = line.split("=")(1)
+ ipPromise.success(ip)
+ }
+
+ outStream.write(line + "\n")
+ outStream.flush()
+ }
+
+ dockerCmd.run(ProcessLogger(findIpAndLog _))
+ val ip = Await.result(ipPromise.future, 30 seconds)
+ val dockerId = Docker.getLastProcessId
+ (ip, dockerId, outFile)
+ }
+}
+
+private[spark] class DockerId(val id: String) {
+ override def toString = id
+}
+
+private[spark] object Docker extends Logging {
+ def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = {
+ val mountCmd = if (mountDir != "") { " -v " + mountDir } else ""
+
+ val cmd = "docker run %s %s %s".format(mountCmd, imageTag, args)
+ logDebug("Run command: " + cmd)
+ cmd
+ }
+
+ def kill(dockerId: DockerId) : Unit = {
+ "docker kill %s".format(dockerId.id).!
+ }
+
+ def getLastProcessId: DockerId = {
+ var id: String = null
+ "docker ps -l -q".!(ProcessLogger(line => id = line))
+ new DockerId(id)
+ }
+} \ No newline at end of file
diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
index a6be8efef1..e607b8c6f4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala
@@ -41,10 +41,13 @@ private[spark] object JsonProtocol {
("starttime" -> obj.startTime) ~
("id" -> obj.id) ~
("name" -> obj.desc.name) ~
+ ("appuiurl" -> obj.appUiUrl) ~
("cores" -> obj.desc.maxCores) ~
("user" -> obj.desc.user) ~
("memoryperslave" -> obj.desc.memoryPerSlave) ~
- ("submitdate" -> obj.submitDate.toString)
+ ("submitdate" -> obj.submitDate.toString) ~
+ ("state" -> obj.state.toString) ~
+ ("duration" -> obj.duration)
}
def writeApplicationDescription(obj: ApplicationDescription) = {
@@ -62,14 +65,15 @@ private[spark] object JsonProtocol {
}
def writeMasterState(obj: MasterStateResponse) = {
- ("url" -> ("spark://" + obj.uri)) ~
+ ("url" -> obj.uri) ~
("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~
("cores" -> obj.workers.map(_.cores).sum) ~
("coresused" -> obj.workers.map(_.coresUsed).sum) ~
("memory" -> obj.workers.map(_.memory).sum) ~
("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~
("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~
- ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo))
+ ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) ~
+ ("status" -> obj.status.toString)
}
def writeWorkerState(obj: WorkerStateResponse) = {
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 78e3747ad8..308a2bfa22 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -39,22 +39,23 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
private val masterActorSystems = ArrayBuffer[ActorSystem]()
private val workerActorSystems = ArrayBuffer[ActorSystem]()
- def start(): String = {
+ def start(): Array[String] = {
logInfo("Starting a local Spark cluster with " + numWorkers + " workers.")
/* Start the Master */
- val (masterSystem, masterPort) = Master.startSystemAndActor(localHostname, 0, 0)
+ val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0)
masterActorSystems += masterSystem
val masterUrl = "spark://" + localHostname + ":" + masterPort
+ val masters = Array(masterUrl)
/* Start the Workers */
for (workerNum <- 1 to numWorkers) {
val (workerSystem, _) = Worker.startSystemAndActor(localHostname, 0, 0, coresPerWorker,
- memoryPerWorker, masterUrl, null, Some(workerNum))
+ memoryPerWorker, masters, null, Some(workerNum))
workerActorSystems += workerSystem
}
- return masterUrl
+ return masters
}
def stop() {
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 0a5f4c368f..993ba6bd3d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -16,6 +16,9 @@
*/
package org.apache.spark.deploy
+
+import com.google.common.collect.MapMaker
+
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.JobConf
@@ -24,11 +27,16 @@ import org.apache.hadoop.mapred.JobConf
* Contains util methods to interact with Hadoop from spark.
*/
class SparkHadoopUtil {
+ // A general, soft-reference map for metadata needed during HadoopRDD split computation
+ // (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
+ private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]()
- // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems
+ // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop
+ // subsystems
def newConfiguration(): Configuration = new Configuration()
- // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster
+ // Add any user credentials to the job conf which are necessary for running on a secure Hadoop
+ // cluster
def addCredentials(conf: JobConf) {}
def isYarnMode(): Boolean = { false }
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 a342dd724a..77422f61ec 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,6 +23,7 @@ import akka.actor._
import akka.actor.Terminated
import akka.pattern.ask
import akka.util.Duration
+import akka.util.duration._
import akka.remote.RemoteClientDisconnected
import akka.remote.RemoteClientLifeCycleEvent
import akka.remote.RemoteClientShutdown
@@ -37,41 +38,81 @@ import org.apache.spark.deploy.master.Master
/**
* The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
* and a listener for cluster events, and calls back the listener when various events occur.
+ *
+ * @param masterUrls Each url should look like spark://host:port.
*/
private[spark] class Client(
actorSystem: ActorSystem,
- masterUrl: String,
+ masterUrls: Array[String],
appDescription: ApplicationDescription,
listener: ClientListener)
extends Logging {
+ val REGISTRATION_TIMEOUT = 20.seconds
+ val REGISTRATION_RETRIES = 3
+
var actor: ActorRef = null
var appId: String = null
+ var registered = false
+ var activeMasterUrl: String = null
class ClientActor extends Actor with Logging {
var master: ActorRef = null
var masterAddress: Address = null
var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times
+ var alreadyDead = false // To avoid calling listener.dead() multiple times
override def preStart() {
- logInfo("Connecting to master " + masterUrl)
try {
- master = context.actorFor(Master.toAkkaUrl(masterUrl))
- masterAddress = master.path.address
- master ! RegisterApplication(appDescription)
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
- context.watch(master) // Doesn't work with remote actors, but useful for testing
+ registerWithMaster()
} catch {
case e: Exception =>
- logError("Failed to connect to master", e)
+ logWarning("Failed to connect to master", e)
markDisconnected()
context.stop(self)
}
}
+ def tryRegisterAllMasters() {
+ for (masterUrl <- masterUrls) {
+ logInfo("Connecting to master " + masterUrl + "...")
+ val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+ actor ! RegisterApplication(appDescription)
+ }
+ }
+
+ def registerWithMaster() {
+ tryRegisterAllMasters()
+
+ var retries = 0
+ lazy val retryTimer: Cancellable =
+ context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
+ retries += 1
+ if (registered) {
+ retryTimer.cancel()
+ } else if (retries >= REGISTRATION_RETRIES) {
+ logError("All masters are unresponsive! Giving up.")
+ markDead()
+ } else {
+ tryRegisterAllMasters()
+ }
+ }
+ retryTimer // start timer
+ }
+
+ def changeMaster(url: String) {
+ activeMasterUrl = url
+ master = context.actorFor(Master.toAkkaUrl(url))
+ masterAddress = master.path.address
+ context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.watch(master) // Doesn't work with remote actors, but useful for testing
+ }
+
override def receive = {
- case RegisteredApplication(appId_) =>
+ case RegisteredApplication(appId_, masterUrl) =>
appId = appId_
+ registered = true
+ changeMaster(masterUrl)
listener.connected(appId)
case ApplicationRemoved(message) =>
@@ -92,23 +133,27 @@ private[spark] class Client(
listener.executorRemoved(fullId, message.getOrElse(""), exitStatus)
}
+ case MasterChanged(masterUrl, masterWebUiUrl) =>
+ logInfo("Master has changed, new master is at " + masterUrl)
+ context.unwatch(master)
+ changeMaster(masterUrl)
+ alreadyDisconnected = false
+ sender ! MasterChangeAcknowledged(appId)
+
case Terminated(actor_) if actor_ == master =>
- logError("Connection to master failed; stopping client")
+ logWarning("Connection to master failed; waiting for master to reconnect...")
markDisconnected()
- context.stop(self)
case RemoteClientDisconnected(transport, address) if address == masterAddress =>
- logError("Connection to master failed; stopping client")
+ logWarning("Connection to master failed; waiting for master to reconnect...")
markDisconnected()
- context.stop(self)
case RemoteClientShutdown(transport, address) if address == masterAddress =>
- logError("Connection to master failed; stopping client")
+ logWarning("Connection to master failed; waiting for master to reconnect...")
markDisconnected()
- context.stop(self)
case StopClient =>
- markDisconnected()
+ markDead()
sender ! true
context.stop(self)
}
@@ -122,6 +167,13 @@ private[spark] class Client(
alreadyDisconnected = true
}
}
+
+ def markDead() {
+ if (!alreadyDead) {
+ listener.dead()
+ alreadyDead = true
+ }
+ }
}
def start() {
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
index 4605368c11..be7a11bd15 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/ClientListener.scala
@@ -27,8 +27,12 @@ package org.apache.spark.deploy.client
private[spark] trait ClientListener {
def connected(appId: String): Unit
+ /** Disconnection may be a temporary state, as we fail over to a new Master. */
def disconnected(): Unit
+ /** Dead means that we couldn't find any Masters to connect to, and have given up. */
+ def dead(): Unit
+
def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int): Unit
def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]): Unit
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
index d5e9a0e095..5b62d3ba6c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
@@ -33,6 +33,11 @@ private[spark] object TestClient {
System.exit(0)
}
+ def dead() {
+ logInfo("Could not connect to master")
+ System.exit(0)
+ }
+
def executorAdded(id: String, workerId: String, hostPort: String, cores: Int, memory: Int) {}
def executorRemoved(id: String, message: String, exitStatus: Option[Int]) {}
@@ -44,7 +49,7 @@ private[spark] object TestClient {
val desc = new ApplicationDescription(
"TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
val listener = new TestListener
- val client = new Client(actorSystem, url, desc, listener)
+ val client = new Client(actorSystem, Array(url), desc, listener)
client.start()
actorSystem.awaitTermination()
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
index bd5327627a..5150b7c7de 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala
@@ -29,23 +29,46 @@ private[spark] class ApplicationInfo(
val submitDate: Date,
val driver: ActorRef,
val appUiUrl: String)
-{
- var state = ApplicationState.WAITING
- var executors = new mutable.HashMap[Int, ExecutorInfo]
- var coresGranted = 0
- var endTime = -1L
- val appSource = new ApplicationSource(this)
-
- private var nextExecutorId = 0
-
- def newExecutorId(): Int = {
- val id = nextExecutorId
- nextExecutorId += 1
- id
+ extends Serializable {
+
+ @transient var state: ApplicationState.Value = _
+ @transient var executors: mutable.HashMap[Int, ExecutorInfo] = _
+ @transient var coresGranted: Int = _
+ @transient var endTime: Long = _
+ @transient var appSource: ApplicationSource = _
+
+ @transient private var nextExecutorId: Int = _
+
+ init()
+
+ private def readObject(in: java.io.ObjectInputStream) : Unit = {
+ in.defaultReadObject()
+ init()
+ }
+
+ private def init() {
+ state = ApplicationState.WAITING
+ executors = new mutable.HashMap[Int, ExecutorInfo]
+ coresGranted = 0
+ endTime = -1L
+ appSource = new ApplicationSource(this)
+ nextExecutorId = 0
+ }
+
+ private def newExecutorId(useID: Option[Int] = None): Int = {
+ useID match {
+ case Some(id) =>
+ nextExecutorId = math.max(nextExecutorId, id + 1)
+ id
+ case None =>
+ val id = nextExecutorId
+ nextExecutorId += 1
+ id
+ }
}
- def addExecutor(worker: WorkerInfo, cores: Int): ExecutorInfo = {
- val exec = new ExecutorInfo(newExecutorId(), this, worker, cores, desc.memoryPerSlave)
+ def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorInfo = {
+ val exec = new ExecutorInfo(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave)
executors(exec.id) = exec
coresGranted += cores
exec
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
index 5a24042e14..c87b66f047 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala
@@ -34,7 +34,7 @@ class ApplicationSource(val application: ApplicationInfo) extends Source {
override def getValue: Long = application.duration
})
- metricRegistry.register(MetricRegistry.name("cores", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("cores"), new Gauge[Int] {
override def getValue: Int = application.coresGranted
})
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..fedf879eff 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,11 +18,11 @@
package org.apache.spark.deploy.master
private[spark] object ApplicationState
- extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
+ extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED", "UNKNOWN") {
type ApplicationState = Value
- val WAITING, RUNNING, FINISHED, FAILED = Value
+ val WAITING, RUNNING, FINISHED, FAILED, UNKNOWN = Value
val MAX_NUM_RETRY = 10
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
index cf384a985e..76db61dd61 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala
@@ -17,7 +17,7 @@
package org.apache.spark.deploy.master
-import org.apache.spark.deploy.ExecutorState
+import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
private[spark] class ExecutorInfo(
val id: Int,
@@ -28,5 +28,10 @@ private[spark] class ExecutorInfo(
var state = ExecutorState.LAUNCHING
+ /** Copy all state (non-val) variables from the given on-the-wire ExecutorDescription. */
+ def copyState(execDesc: ExecutorDescription) {
+ state = execDesc.state
+ }
+
def fullId: String = application.id + "/" + id
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
new file mode 100644
index 0000000000..c0849ef324
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -0,0 +1,90 @@
+/*
+ * 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.master
+
+import java.io._
+
+import scala.Serializable
+
+import akka.serialization.Serialization
+import org.apache.spark.Logging
+
+/**
+ * Stores data in a single on-disk directory with one file per application and worker.
+ * Files are deleted when applications and workers are removed.
+ *
+ * @param dir Directory to store files. Created if non-existent (but not recursively).
+ * @param serialization Used to serialize our objects.
+ */
+private[spark] class FileSystemPersistenceEngine(
+ val dir: String,
+ val serialization: Serialization)
+ extends PersistenceEngine with Logging {
+
+ new File(dir).mkdir()
+
+ override def addApplication(app: ApplicationInfo) {
+ val appFile = new File(dir + File.separator + "app_" + app.id)
+ serializeIntoFile(appFile, app)
+ }
+
+ override def removeApplication(app: ApplicationInfo) {
+ new File(dir + File.separator + "app_" + app.id).delete()
+ }
+
+ override def addWorker(worker: WorkerInfo) {
+ val workerFile = new File(dir + File.separator + "worker_" + worker.id)
+ serializeIntoFile(workerFile, worker)
+ }
+
+ override def removeWorker(worker: WorkerInfo) {
+ new File(dir + File.separator + "worker_" + worker.id).delete()
+ }
+
+ override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+ val sortedFiles = new File(dir).listFiles().sortBy(_.getName)
+ val appFiles = sortedFiles.filter(_.getName.startsWith("app_"))
+ val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+ val workerFiles = sortedFiles.filter(_.getName.startsWith("worker_"))
+ val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
+ (apps, workers)
+ }
+
+ private def serializeIntoFile(file: File, value: Serializable) {
+ val created = file.createNewFile()
+ if (!created) { throw new IllegalStateException("Could not create file: " + file) }
+
+ val serializer = serialization.findSerializerFor(value)
+ val serialized = serializer.toBinary(value)
+
+ val out = new FileOutputStream(file)
+ out.write(serialized)
+ out.close()
+ }
+
+ def deserializeFromFile[T <: Serializable](file: File)(implicit m: Manifest[T]): T = {
+ val fileData = new Array[Byte](file.length().asInstanceOf[Int])
+ val dis = new DataInputStream(new FileInputStream(file))
+ dis.readFully(fileData)
+ dis.close()
+
+ val clazz = m.erasure.asInstanceOf[Class[T]]
+ val serializer = serialization.serializerFor(clazz)
+ serializer.fromBinary(fileData).asInstanceOf[T]
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
new file mode 100644
index 0000000000..f25a1ad3bf
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala
@@ -0,0 +1,45 @@
+/*
+ * 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.master
+
+import akka.actor.{Actor, ActorRef}
+
+import org.apache.spark.deploy.master.MasterMessages.ElectedLeader
+
+/**
+ * A LeaderElectionAgent keeps track of whether the current Master is the leader, meaning it
+ * is the only Master serving requests.
+ * In addition to the API provided, the LeaderElectionAgent will use of the following messages
+ * to inform the Master of leader changes:
+ * [[org.apache.spark.deploy.master.MasterMessages.ElectedLeader ElectedLeader]]
+ * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]]
+ */
+private[spark] trait LeaderElectionAgent extends Actor {
+ val masterActor: ActorRef
+}
+
+/** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */
+private[spark] class MonarchyLeaderAgent(val masterActor: ActorRef) extends LeaderElectionAgent {
+ override def preStart() {
+ masterActor ! ElectedLeader
+ }
+
+ override def receive = {
+ case _ =>
+ }
+}
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 7cf0a7754f..cd916672ac 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,30 +17,36 @@
package org.apache.spark.deploy.master
-import java.text.SimpleDateFormat
import java.util.Date
+import java.text.SimpleDateFormat
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import akka.actor._
import akka.actor.Terminated
+import akka.dispatch.Await
+import akka.pattern.ask
import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
+import akka.serialization.SerializationExtension
import akka.util.duration._
+import akka.util.{Duration, Timeout}
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
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 {
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
val REAPER_ITERATIONS = System.getProperty("spark.dead.worker.persistence", "15").toInt
-
+ val RECOVERY_DIR = System.getProperty("spark.deploy.recoveryDirectory", "")
+ val RECOVERY_MODE = System.getProperty("spark.deploy.recoveryMode", "NONE")
+
var nextAppNumber = 0
val workers = new HashSet[WorkerInfo]
val idToWorker = new HashMap[String, WorkerInfo]
@@ -70,51 +76,116 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
if (envVar != null) envVar else host
}
+ val masterUrl = "spark://" + host + ":" + port
+ var masterWebUiUrl: String = _
+
+ var state = RecoveryState.STANDBY
+
+ var persistenceEngine: PersistenceEngine = _
+
+ var leaderElectionAgent: ActorRef = _
+
// As a temporary workaround before better ways of configuring memory, we allow users to set
// a flag that will perform round-robin scheduling across the nodes (spreading out each app
// among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "true").toBoolean
override def preStart() {
- logInfo("Starting Spark master at spark://" + host + ":" + port)
+ 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[RemoteClientLifeCycleEvent])
webUi.start()
+ masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get
context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut)
masterMetricsSystem.registerSource(masterSource)
masterMetricsSystem.start()
applicationMetricsSystem.start()
+
+ persistenceEngine = RECOVERY_MODE match {
+ case "ZOOKEEPER" =>
+ logInfo("Persisting recovery state to ZooKeeper")
+ new ZooKeeperPersistenceEngine(SerializationExtension(context.system))
+ case "FILESYSTEM" =>
+ logInfo("Persisting recovery state to directory: " + RECOVERY_DIR)
+ new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system))
+ case _ =>
+ new BlackHolePersistenceEngine()
+ }
+
+ leaderElectionAgent = context.actorOf(Props(
+ RECOVERY_MODE match {
+ case "ZOOKEEPER" =>
+ new ZooKeeperLeaderElectionAgent(self, masterUrl)
+ case _ =>
+ new MonarchyLeaderAgent(self)
+ }))
+ }
+
+ override def preRestart(reason: Throwable, message: Option[Any]) {
+ super.preRestart(reason, message) // calls postStop()!
+ logError("Master actor restarted due to exception", reason)
}
override def postStop() {
webUi.stop()
masterMetricsSystem.stop()
applicationMetricsSystem.stop()
+ persistenceEngine.close()
+ context.stop(leaderElectionAgent)
}
override def receive = {
- case RegisterWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress) => {
+ case ElectedLeader => {
+ val (storedApps, storedWorkers) = persistenceEngine.readPersistedData()
+ state = if (storedApps.isEmpty && storedWorkers.isEmpty)
+ 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() }
+ }
+ }
+
+ case RevokedLeadership => {
+ logError("Leadership has been revoked -- master shutting down.")
+ System.exit(0)
+ }
+
+ case RegisterWorker(id, host, workerPort, cores, memory, webUiPort, publicAddress) => {
logInfo("Registering worker %s:%d with %d cores, %s RAM".format(
host, workerPort, cores, Utils.megabytesToString(memory)))
- if (idToWorker.contains(id)) {
+ if (state == RecoveryState.STANDBY) {
+ // ignore, don't send response
+ } else if (idToWorker.contains(id)) {
sender ! RegisterWorkerFailed("Duplicate worker ID")
} else {
- addWorker(id, host, workerPort, cores, memory, worker_webUiPort, publicAddress)
+ val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
+ registerWorker(worker)
context.watch(sender) // This doesn't work with remote actors but helps for testing
- sender ! RegisteredWorker("http://" + masterPublicAddress + ":" + webUi.boundPort.get)
+ persistenceEngine.addWorker(worker)
+ sender ! RegisteredWorker(masterUrl, masterWebUiUrl)
schedule()
}
}
case RegisterApplication(description) => {
- logInfo("Registering app " + description.name)
- val app = addApplication(description, sender)
- logInfo("Registered app " + description.name + " with ID " + app.id)
- waitingApps += app
- context.watch(sender) // This doesn't work with remote actors but helps for testing
- sender ! RegisteredApplication(app.id)
- schedule()
+ if (state == RecoveryState.STANDBY) {
+ // ignore, don't send response
+ } else {
+ logInfo("Registering app " + description.name)
+ val app = createApplication(description, sender)
+ registerApplication(app)
+ logInfo("Registered app " + description.name + " with ID " + app.id)
+ context.watch(sender) // This doesn't work with remote actors but helps for testing
+ persistenceEngine.addApplication(app)
+ sender ! RegisteredApplication(app.id, masterUrl)
+ schedule()
+ }
}
case ExecutorStateChanged(appId, execId, state, message, exitStatus) => {
@@ -154,32 +225,116 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}
}
+ case MasterChangeAcknowledged(appId) => {
+ idToApp.get(appId) match {
+ case Some(app) =>
+ logInfo("Application has been re-registered: " + appId)
+ app.state = ApplicationState.WAITING
+ case None =>
+ logWarning("Master change ack from unknown app: " + appId)
+ }
+
+ if (canCompleteRecovery) { completeRecovery() }
+ }
+
+ case WorkerSchedulerStateResponse(workerId, executors) => {
+ idToWorker.get(workerId) match {
+ case Some(worker) =>
+ logInfo("Worker has been re-registered: " + workerId)
+ worker.state = WorkerState.ALIVE
+
+ val validExecutors = executors.filter(exec => idToApp.get(exec.appId).isDefined)
+ for (exec <- validExecutors) {
+ val app = idToApp.get(exec.appId).get
+ val execInfo = app.addExecutor(worker, exec.cores, Some(exec.execId))
+ worker.addExecutor(execInfo)
+ execInfo.copyState(exec)
+ }
+ case None =>
+ logWarning("Scheduler state from unknown worker: " + workerId)
+ }
+
+ 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
actorToWorker.get(actor).foreach(removeWorker)
actorToApp.get(actor).foreach(finishApplication)
+ if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
}
case RemoteClientDisconnected(transport, 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)
+ if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
}
case RemoteClientShutdown(transport, 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)
+ if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
}
case RequestMasterState => {
- sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray)
+ sender ! MasterStateResponse(host, port, workers.toArray, apps.toArray, completedApps.toArray,
+ state)
}
case CheckForWorkerTimeOut => {
timeOutDeadWorkers()
}
+
+ case RequestWebUIPort => {
+ sender ! WebUIPortResponse(webUi.boundPort.getOrElse(-1))
+ }
+ }
+
+ def canCompleteRecovery =
+ workers.count(_.state == WorkerState.UNKNOWN) == 0 &&
+ apps.count(_.state == ApplicationState.UNKNOWN) == 0
+
+ def beginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo]) {
+ for (app <- storedApps) {
+ logInfo("Trying to recover app: " + app.id)
+ try {
+ registerApplication(app)
+ app.state = ApplicationState.UNKNOWN
+ app.driver ! MasterChanged(masterUrl, masterWebUiUrl)
+ } catch {
+ case e: Exception => logInfo("App " + app.id + " had exception on reconnect")
+ }
+ }
+
+ for (worker <- storedWorkers) {
+ logInfo("Trying to recover worker: " + worker.id)
+ try {
+ registerWorker(worker)
+ worker.state = WorkerState.UNKNOWN
+ worker.actor ! MasterChanged(masterUrl, masterWebUiUrl)
+ } catch {
+ case e: Exception => logInfo("Worker " + worker.id + " had exception on reconnect")
+ }
+ }
+ }
+
+ def completeRecovery() {
+ // Ensure "only-once" recovery semantics using a short synchronization period.
+ synchronized {
+ if (state != RecoveryState.RECOVERING) { return }
+ state = RecoveryState.COMPLETING_RECOVERY
+ }
+
+ // Kill off any workers and apps that didn't respond to us.
+ workers.filter(_.state == WorkerState.UNKNOWN).foreach(removeWorker)
+ apps.filter(_.state == ApplicationState.UNKNOWN).foreach(finishApplication)
+
+ state = RecoveryState.ALIVE
+ schedule()
+ logInfo("Recovery complete - resuming operations!")
}
/**
@@ -196,6 +351,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
* every time a new app joins or resource availability changes.
*/
def schedule() {
+ if (state != RecoveryState.ALIVE) { return }
// Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
// in the queue, then the second app, etc.
if (spreadOutApps) {
@@ -243,14 +399,13 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) {
logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
worker.addExecutor(exec)
- worker.actor ! LaunchExecutor(
+ worker.actor ! LaunchExecutor(masterUrl,
exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome)
exec.application.driver ! ExecutorAdded(
exec.id, worker.id, worker.hostPort, exec.cores, exec.memory)
}
- def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int,
- publicAddress: String): WorkerInfo = {
+ def registerWorker(worker: WorkerInfo): Unit = {
// There may be one or more refs to dead workers on this same node (w/ different ID's),
// remove them.
workers.filter { w =>
@@ -258,12 +413,17 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
}.foreach { w =>
workers -= w
}
- val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
+
+ val workerAddress = worker.actor.path.address
+ if (addressToWorker.contains(workerAddress)) {
+ logInfo("Attempted to re-register worker at same address: " + workerAddress)
+ return
+ }
+
workers += worker
idToWorker(worker.id) = worker
- actorToWorker(sender) = worker
- addressToWorker(sender.path.address) = worker
- worker
+ actorToWorker(worker.actor) = worker
+ addressToWorker(workerAddress) = worker
}
def removeWorker(worker: WorkerInfo) {
@@ -278,25 +438,36 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
exec.id, ExecutorState.LOST, Some("worker lost"), None)
exec.application.removeExecutor(exec)
}
+ persistenceEngine.removeWorker(worker)
}
- def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
+ def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
val now = System.currentTimeMillis()
val date = new Date(now)
- val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+ new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+ }
+
+ def registerApplication(app: ApplicationInfo): Unit = {
+ val appAddress = app.driver.path.address
+ if (addressToWorker.contains(appAddress)) {
+ logInfo("Attempted to re-register application at same address: " + appAddress)
+ return
+ }
+
applicationMetricsSystem.registerSource(app.appSource)
apps += app
idToApp(app.id) = app
- actorToApp(driver) = app
- addressToApp(driver.path.address) = 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 >= desc.memoryPerSlave)) {
+ if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= app.desc.memoryPerSlave)) {
logWarning("Could not find any workers with enough memory for " + firstApp.get.id)
}
- app
+ waitingApps += app
}
def finishApplication(app: ApplicationInfo) {
@@ -321,13 +492,14 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
waitingApps -= app
for (exec <- app.executors.values) {
exec.worker.removeExecutor(exec)
- exec.worker.actor ! KillExecutor(exec.application.id, exec.id)
+ exec.worker.actor ! KillExecutor(masterUrl, exec.application.id, exec.id)
exec.state = ExecutorState.KILLED
}
app.markFinished(state)
if (state != ApplicationState.FINISHED) {
app.driver ! ApplicationRemoved(state.toString)
}
+ persistenceEngine.removeApplication(app)
schedule()
}
}
@@ -364,7 +536,7 @@ private[spark] object Master {
def main(argStrings: Array[String]) {
val args = new MasterArguments(argStrings)
- val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort)
+ val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort)
actorSystem.awaitTermination()
}
@@ -378,9 +550,14 @@ private[spark] object Master {
}
}
- def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int) = {
+ 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)
- (actorSystem, boundPort)
+ val timeoutDuration = Duration.create(
+ System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+ implicit val timeout = Timeout(timeoutDuration)
+ val respFuture = actor ? RequestWebUIPort // ask pattern
+ val resp = Await.result(respFuture, timeoutDuration).asInstanceOf[WebUIPortResponse]
+ (actorSystem, boundPort, resp.webUIBoundPort)
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
new file mode 100644
index 0000000000..74a9f8cd82
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.master
+
+sealed trait MasterMessages extends Serializable
+
+/** Contains messages seen only by the Master and its associated entities. */
+private[master] object MasterMessages {
+
+ // LeaderElectionAgent to Master
+
+ case object ElectedLeader
+
+ case object RevokedLeadership
+
+ // Actor System to LeaderElectionAgent
+
+ case object CheckLeader
+
+ // Actor System to Master
+
+ case object CheckForWorkerTimeOut
+
+ case class BeginRecovery(storedApps: Seq[ApplicationInfo], storedWorkers: Seq[WorkerInfo])
+
+ case object CompleteRecovery
+
+ case object RequestWebUIPort
+
+ case class WebUIPortResponse(webUIBoundPort: Int)
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
index 23d1cb77da..36c1b87b7f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
@@ -26,17 +26,17 @@ private[spark] class MasterSource(val master: Master) extends Source {
val sourceName = "master"
// Gauge for worker numbers in cluster
- metricRegistry.register(MetricRegistry.name("workers","number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("workers"), new Gauge[Int] {
override def getValue: Int = master.workers.size
})
// Gauge for application numbers in cluster
- metricRegistry.register(MetricRegistry.name("apps", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("apps"), new Gauge[Int] {
override def getValue: Int = master.apps.size
})
// Gauge for waiting application numbers in cluster
- metricRegistry.register(MetricRegistry.name("waitingApps", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("waitingApps"), new Gauge[Int] {
override def getValue: Int = master.waitingApps.size
})
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
new file mode 100644
index 0000000000..94b986caf2
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.master
+
+/**
+ * Allows Master to persist any state that is necessary in order to recover from a failure.
+ * The following semantics are required:
+ * - addApplication and addWorker are called before completing registration of a new app/worker.
+ * - removeApplication and removeWorker are called at any time.
+ * Given these two requirements, we will have all apps and workers persisted, but
+ * we might not have yet deleted apps or workers that finished (so their liveness must be verified
+ * during recovery).
+ */
+private[spark] trait PersistenceEngine {
+ def addApplication(app: ApplicationInfo)
+
+ def removeApplication(app: ApplicationInfo)
+
+ def addWorker(worker: WorkerInfo)
+
+ def removeWorker(worker: WorkerInfo)
+
+ /**
+ * Returns the persisted data sorted by their respective ids (which implies that they're
+ * sorted by time of creation).
+ */
+ def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo])
+
+ def close() {}
+}
+
+private[spark] class BlackHolePersistenceEngine extends PersistenceEngine {
+ override def addApplication(app: ApplicationInfo) {}
+ override def removeApplication(app: ApplicationInfo) {}
+ override def addWorker(worker: WorkerInfo) {}
+ override def removeWorker(worker: WorkerInfo) {}
+ override def readPersistedData() = (Nil, Nil)
+}
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
new file mode 100644
index 0000000000..b91be821f0
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
@@ -0,0 +1,26 @@
+/*
+ * 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.master
+
+private[spark] object RecoveryState
+ extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") {
+
+ type MasterState = Value
+
+ val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value
+}
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
new file mode 100644
index 0000000000..81e15c534f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
@@ -0,0 +1,203 @@
+/*
+ * 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.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
+
+/**
+ * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry
+ * logic. If the ZooKeeper session expires or otherwise dies, a new ZooKeeper session will be
+ * created. If ZooKeeper remains down after several retries, the given
+ * [[org.apache.spark.deploy.master.SparkZooKeeperWatcher SparkZooKeeperWatcher]] will be
+ * 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").
+ */
+private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging {
+ val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "")
+
+ val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE
+ val ZK_TIMEOUT_MILLIS = 30000
+ val RETRY_WAIT_MILLIS = 5000
+ val ZK_CHECK_PERIOD_MILLIS = 10000
+ val MAX_RECONNECT_ATTEMPTS = 3
+
+ private var zk: ZooKeeper = _
+
+ private val watcher = new ZooKeeperWatcher()
+ private var reconnectAttempts = 0
+ private var closed = false
+
+ /** Connect to ZooKeeper to start the session. Must be called before anything else. */
+ def connect() {
+ connectToZooKeeper()
+
+ new Thread() {
+ override def run() = sessionMonitorThread()
+ }.start()
+ }
+
+ def sessionMonitorThread(): Unit = {
+ while (!closed) {
+ Thread.sleep(ZK_CHECK_PERIOD_MILLIS)
+ if (zk.getState != ZooKeeper.States.CONNECTED) {
+ reconnectAttempts += 1
+ val attemptsLeft = MAX_RECONNECT_ATTEMPTS - reconnectAttempts
+ if (attemptsLeft <= 0) {
+ logError("Could not connect to ZooKeeper: system failure")
+ zkWatcher.zkDown()
+ close()
+ } else {
+ logWarning("ZooKeeper connection failed, retrying " + attemptsLeft + " more times...")
+ connectToZooKeeper()
+ }
+ }
+ }
+ }
+
+ def close() {
+ if (!closed && zk != null) { zk.close() }
+ closed = true
+ }
+
+ private def connectToZooKeeper() {
+ if (zk != null) zk.close()
+ zk = new ZooKeeper(ZK_URL, ZK_TIMEOUT_MILLIS, watcher)
+ }
+
+ /**
+ * Attempts to maintain a live ZooKeeper exception despite (very) transient failures.
+ * Mainly useful for handling the natural ZooKeeper session expiration.
+ */
+ private class ZooKeeperWatcher extends Watcher {
+ def process(event: WatchedEvent) {
+ if (closed) { return }
+
+ event.getState match {
+ case KeeperState.SyncConnected =>
+ reconnectAttempts = 0
+ zkWatcher.zkSessionCreated()
+ case KeeperState.Expired =>
+ connectToZooKeeper()
+ case KeeperState.Disconnected =>
+ logWarning("ZooKeeper disconnected, will retry...")
+ }
+ }
+ }
+
+ def create(path: String, bytes: Array[Byte], createMode: CreateMode): String = {
+ retry {
+ zk.create(path, bytes, ZK_ACL, createMode)
+ }
+ }
+
+ def exists(path: String, watcher: Watcher = null): Stat = {
+ retry {
+ zk.exists(path, watcher)
+ }
+ }
+
+ def getChildren(path: String, watcher: Watcher = null): List[String] = {
+ retry {
+ zk.getChildren(path, watcher).toList
+ }
+ }
+
+ def getData(path: String): Array[Byte] = {
+ retry {
+ zk.getData(path, false, null)
+ }
+ }
+
+ def delete(path: String, version: Int = -1): Unit = {
+ retry {
+ zk.delete(path, version)
+ }
+ }
+
+ /**
+ * Creates the given directory (non-recursively) if it doesn't exist.
+ * All znodes are created in PERSISTENT mode with no data.
+ */
+ def mkdir(path: String) {
+ if (exists(path) == null) {
+ try {
+ create(path, "".getBytes, CreateMode.PERSISTENT)
+ } catch {
+ case e: Exception =>
+ // If the exception caused the directory not to be created, bubble it up,
+ // otherwise ignore it.
+ if (exists(path) == null) { throw e }
+ }
+ }
+ }
+
+ /**
+ * Recursively creates all directories up to the given one.
+ * All znodes are created in PERSISTENT mode with no data.
+ */
+ def mkdirRecursive(path: String) {
+ var fullDir = ""
+ for (dentry <- path.split("/").tail) {
+ fullDir += "/" + dentry
+ mkdir(fullDir)
+ }
+ }
+
+ /**
+ * Retries the given function up to 3 times. The assumption is that failure is transient,
+ * UNLESS it is a semantic exception (i.e., trying to get data from a node that doesn't exist),
+ * in which case the exception will be thrown without retries.
+ *
+ * @param fn Block to execute, possibly multiple times.
+ */
+ def retry[T](fn: => T, n: Int = MAX_RECONNECT_ATTEMPTS): T = {
+ try {
+ fn
+ } catch {
+ case e: KeeperException.NoNodeException => throw e
+ case e: KeeperException.NodeExistsException => throw e
+ case e if n > 0 =>
+ logError("ZooKeeper exception, " + n + " more retries...", e)
+ Thread.sleep(RETRY_WAIT_MILLIS)
+ retry(fn, n-1)
+ }
+ }
+}
+
+trait SparkZooKeeperWatcher {
+ /**
+ * Called whenever a ZK session is created --
+ * this will occur when we create our first session as well as each time
+ * the session expires or errors out.
+ */
+ def zkSessionCreated()
+
+ /**
+ * Called if ZK appears to be completely down (i.e., not just a transient error).
+ * We will no longer attempt to reconnect to ZK, and the SparkZooKeeperSession is considered dead.
+ */
+ def zkDown()
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
index 6219f11f2a..e05f587b58 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala
@@ -22,28 +22,44 @@ import scala.collection.mutable
import org.apache.spark.util.Utils
private[spark] class WorkerInfo(
- val id: String,
- val host: String,
- val port: Int,
- val cores: Int,
- val memory: Int,
- val actor: ActorRef,
- val webUiPort: Int,
- val publicAddress: String) {
+ val id: String,
+ val host: String,
+ val port: Int,
+ val cores: Int,
+ val memory: Int,
+ val actor: ActorRef,
+ val webUiPort: Int,
+ val publicAddress: String)
+ extends Serializable {
Utils.checkHost(host, "Expected hostname")
assert (port > 0)
- var executors = new mutable.HashMap[String, ExecutorInfo] // fullId => info
- var state: WorkerState.Value = WorkerState.ALIVE
- var coresUsed = 0
- var memoryUsed = 0
+ @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // fullId => info
+ @transient var state: WorkerState.Value = _
+ @transient var coresUsed: Int = _
+ @transient var memoryUsed: Int = _
- var lastHeartbeat = System.currentTimeMillis()
+ @transient var lastHeartbeat: Long = _
+
+ init()
def coresFree: Int = cores - coresUsed
def memoryFree: Int = memory - memoryUsed
+ private def readObject(in: java.io.ObjectInputStream) : Unit = {
+ in.defaultReadObject()
+ init()
+ }
+
+ private def init() {
+ executors = new mutable.HashMap
+ state = WorkerState.ALIVE
+ coresUsed = 0
+ memoryUsed = 0
+ lastHeartbeat = System.currentTimeMillis()
+ }
+
def hostPort: String = {
assert (port > 0)
host + ":" + port
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..c8d34f25e2 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,8 +17,10 @@
package org.apache.spark.deploy.master
-private[spark] object WorkerState extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED") {
+private[spark] object WorkerState
+ extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED", "UNKNOWN") {
+
type WorkerState = Value
- val ALIVE, DEAD, DECOMMISSIONED = Value
+ val ALIVE, DEAD, DECOMMISSIONED, UNKNOWN = Value
}
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
new file mode 100644
index 0000000000..7809013e83
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
@@ -0,0 +1,136 @@
+/*
+ * 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.master
+
+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
+
+private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String)
+ extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging {
+
+ val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
+
+ private val watcher = new ZooKeeperWatcher()
+ private val zk = new SparkZooKeeperSession(this)
+ private var status = LeadershipStatus.NOT_LEADER
+ private var myLeaderFile: String = _
+ private var leaderUrl: String = _
+
+ override def preStart() {
+ logInfo("Starting ZooKeeper LeaderElection agent")
+ zk.connect()
+ }
+
+ override def zkSessionCreated() {
+ synchronized {
+ zk.mkdirRecursive(WORKING_DIR)
+ myLeaderFile =
+ zk.create(WORKING_DIR + "/master_", masterUrl.getBytes, CreateMode.EPHEMERAL_SEQUENTIAL)
+ self ! CheckLeader
+ }
+ }
+
+ override def preRestart(reason: scala.Throwable, message: scala.Option[scala.Any]) {
+ logError("LeaderElectionAgent failed, waiting " + zk.ZK_TIMEOUT_MILLIS + "...", reason)
+ Thread.sleep(zk.ZK_TIMEOUT_MILLIS)
+ super.preRestart(reason, message)
+ }
+
+ override def zkDown() {
+ logError("ZooKeeper down! LeaderElectionAgent shutting down Master.")
+ System.exit(1)
+ }
+
+ override def postStop() {
+ zk.close()
+ }
+
+ override def receive = {
+ case CheckLeader => checkLeader()
+ }
+
+ private class ZooKeeperWatcher extends Watcher {
+ def process(event: WatchedEvent) {
+ if (event.getType == EventType.NodeDeleted) {
+ logInfo("Leader file disappeared, a master is down!")
+ self ! CheckLeader
+ }
+ }
+ }
+
+ /** Uses ZK leader election. Navigates several ZK potholes along the way. */
+ def checkLeader() {
+ val masters = zk.getChildren(WORKING_DIR).toList
+ val leader = masters.sorted.head
+ val leaderFile = WORKING_DIR + "/" + leader
+
+ // Setup a watch for the current leader.
+ zk.exists(leaderFile, watcher)
+
+ try {
+ leaderUrl = new String(zk.getData(leaderFile))
+ } catch {
+ // A NoNodeException may be thrown if old leader died since the start of this method call.
+ // This is fine -- just check again, since we're guaranteed to see the new values.
+ case e: KeeperException.NoNodeException =>
+ logInfo("Leader disappeared while reading it -- finding next leader")
+ checkLeader()
+ return
+ }
+
+ // Synchronization used to ensure no interleaving between the creation of a new session and the
+ // checking of a leader, which could cause us to delete our real leader file erroneously.
+ synchronized {
+ val isLeader = myLeaderFile == leaderFile
+ if (!isLeader && leaderUrl == masterUrl) {
+ // 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.
+ // 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
+ // leader changes.
+ assert(leaderFile < myLeaderFile)
+ logWarning("Cleaning up old ZK master election file that points to this master.")
+ zk.delete(leaderFile)
+ } else {
+ updateLeadershipStatus(isLeader)
+ }
+ }
+ }
+
+ def updateLeadershipStatus(isLeader: Boolean) {
+ if (isLeader && status == LeadershipStatus.NOT_LEADER) {
+ status = LeadershipStatus.LEADER
+ masterActor ! ElectedLeader
+ } else if (!isLeader && status == LeadershipStatus.LEADER) {
+ status = LeadershipStatus.NOT_LEADER
+ masterActor ! RevokedLeadership
+ }
+ }
+
+ private object LeadershipStatus extends Enumeration {
+ type LeadershipStatus = Value
+ val LEADER, NOT_LEADER = Value
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
new file mode 100644
index 0000000000..a0233a7271
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -0,0 +1,85 @@
+/*
+ * 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.master
+
+import org.apache.spark.Logging
+import org.apache.zookeeper._
+
+import akka.serialization.Serialization
+
+class ZooKeeperPersistenceEngine(serialization: Serialization)
+ extends PersistenceEngine
+ with SparkZooKeeperWatcher
+ with Logging
+{
+ val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
+
+ val zk = new SparkZooKeeperSession(this)
+
+ zk.connect()
+
+ override def zkSessionCreated() {
+ zk.mkdirRecursive(WORKING_DIR)
+ }
+
+ override def zkDown() {
+ logError("PersistenceEngine disconnected from ZooKeeper -- ZK looks down.")
+ }
+
+ override def addApplication(app: ApplicationInfo) {
+ serializeIntoFile(WORKING_DIR + "/app_" + app.id, app)
+ }
+
+ override def removeApplication(app: ApplicationInfo) {
+ zk.delete(WORKING_DIR + "/app_" + app.id)
+ }
+
+ override def addWorker(worker: WorkerInfo) {
+ serializeIntoFile(WORKING_DIR + "/worker_" + worker.id, worker)
+ }
+
+ override def removeWorker(worker: WorkerInfo) {
+ zk.delete(WORKING_DIR + "/worker_" + worker.id)
+ }
+
+ override def close() {
+ zk.close()
+ }
+
+ override def readPersistedData(): (Seq[ApplicationInfo], Seq[WorkerInfo]) = {
+ val sortedFiles = zk.getChildren(WORKING_DIR).toList.sorted
+ val appFiles = sortedFiles.filter(_.startsWith("app_"))
+ val apps = appFiles.map(deserializeFromFile[ApplicationInfo])
+ val workerFiles = sortedFiles.filter(_.startsWith("worker_"))
+ val workers = workerFiles.map(deserializeFromFile[WorkerInfo])
+ (apps, workers)
+ }
+
+ private def serializeIntoFile(path: String, value: Serializable) {
+ val serializer = serialization.findSerializerFor(value)
+ val serialized = serializer.toBinary(value)
+ zk.create(path, serialized, CreateMode.PERSISTENT)
+ }
+
+ def deserializeFromFile[T <: Serializable](filename: String)(implicit m: Manifest[T]): T = {
+ val fileData = zk.getData("/spark/master_status/" + filename)
+ val clazz = m.erasure.asInstanceOf[Class[T]]
+ val serializer = serialization.serializerFor(clazz)
+ serializer.fromBinary(fileData).asInstanceOf[T]
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
index e3dc30eefc..8fabc95665 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala
@@ -43,7 +43,8 @@ private[spark] class ExecutorRunner(
val workerId: String,
val host: String,
val sparkHome: File,
- val workDir: File)
+ val workDir: File,
+ var state: ExecutorState.Value)
extends Logging {
val fullId = appId + "/" + execId
@@ -83,7 +84,8 @@ private[spark] class ExecutorRunner(
process.destroy()
process.waitFor()
}
- worker ! ExecutorStateChanged(appId, execId, ExecutorState.KILLED, None, None)
+ state = ExecutorState.KILLED
+ worker ! ExecutorStateChanged(appId, execId, state, None, None)
Runtime.getRuntime.removeShutdownHook(shutdownHook)
}
}
@@ -180,9 +182,9 @@ private[spark] class ExecutorRunner(
// long-lived processes only. However, in the future, we might restart the executor a few
// times on the same machine.
val exitCode = process.waitFor()
+ state = ExecutorState.FAILED
val message = "Command exited with code " + exitCode
- worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message),
- Some(exitCode))
+ worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode))
} catch {
case interrupted: InterruptedException =>
logInfo("Runner thread for executor " + fullId + " interrupted")
@@ -192,8 +194,9 @@ private[spark] class ExecutorRunner(
if (process != null) {
process.destroy()
}
+ state = ExecutorState.FAILED
val message = e.getClass + ": " + e.getMessage
- worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message), None)
+ worker ! ExecutorStateChanged(appId, execId, state, Some(message), None)
}
}
}
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 09530beb3b..216d9d44ac 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,26 +23,28 @@ import java.io.File
import scala.collection.mutable.HashMap
-import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
+import akka.actor._
import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
import akka.util.duration._
-import org.apache.spark.{Logging}
-import org.apache.spark.deploy.ExecutorState
+import org.apache.spark.Logging
+import org.apache.spark.deploy.{ExecutorDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
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}
-
+/**
+ * @param masterUrls Each url should look like spark://host:port.
+ */
private[spark] class Worker(
host: String,
port: Int,
webUiPort: Int,
cores: Int,
memory: Int,
- masterUrl: String,
+ masterUrls: Array[String],
workDirPath: String = null)
extends Actor with Logging {
@@ -54,8 +56,18 @@ private[spark] class Worker(
// Send a heartbeat every (heartbeat timeout) / 4 milliseconds
val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4
+ val REGISTRATION_TIMEOUT = 20.seconds
+ val REGISTRATION_RETRIES = 3
+
+ // Index into masterUrls that we're currently trying to register with.
+ var masterIndex = 0
+
+ val masterLock: Object = new Object()
var master: ActorRef = null
- var masterWebUiUrl : String = ""
+ var activeMasterUrl: String = ""
+ var activeMasterWebUiUrl : String = ""
+ @volatile var registered = false
+ @volatile var connected = false
val workerId = generateWorkerId()
var sparkHome: File = null
var workDir: File = null
@@ -95,6 +107,7 @@ private[spark] class Worker(
}
override def preStart() {
+ assert(!registered)
logInfo("Starting Spark worker %s:%d with %d cores, %s RAM".format(
host, port, cores, Utils.megabytesToString(memory)))
sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse("."))
@@ -103,44 +116,98 @@ private[spark] class Worker(
webUi = new WorkerWebUI(this, workDir, Some(webUiPort))
webUi.start()
- connectToMaster()
+ registerWithMaster()
metricsSystem.registerSource(workerSource)
metricsSystem.start()
}
- def connectToMaster() {
- 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.watch(master) // Doesn't work with remote actors, but useful for testing
+ def changeMaster(url: String, uiUrl: String) {
+ masterLock.synchronized {
+ activeMasterUrl = url
+ activeMasterWebUiUrl = uiUrl
+ master = context.actorFor(Master.toAkkaUrl(activeMasterUrl))
+ context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.watch(master) // Doesn't work with remote actors, but useful for testing
+ connected = true
+ }
+ }
+
+ def tryRegisterAllMasters() {
+ for (masterUrl <- masterUrls) {
+ logInfo("Connecting to master " + masterUrl + "...")
+ val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+ actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get,
+ publicAddress)
+ }
+ }
+
+ def registerWithMaster() {
+ tryRegisterAllMasters()
+
+ var retries = 0
+ lazy val retryTimer: Cancellable =
+ context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
+ retries += 1
+ if (registered) {
+ retryTimer.cancel()
+ } else if (retries >= REGISTRATION_RETRIES) {
+ logError("All masters are unresponsive! Giving up.")
+ System.exit(1)
+ } else {
+ tryRegisterAllMasters()
+ }
+ }
+ retryTimer // start timer
}
override def receive = {
- case RegisteredWorker(url) =>
- masterWebUiUrl = url
- logInfo("Successfully registered with master")
- context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis) {
- master ! Heartbeat(workerId)
+ case RegisteredWorker(masterUrl, masterWebUiUrl) =>
+ logInfo("Successfully registered with master " + masterUrl)
+ registered = true
+ changeMaster(masterUrl, masterWebUiUrl)
+ context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat)
+
+ case SendHeartbeat =>
+ masterLock.synchronized {
+ if (connected) { master ! Heartbeat(workerId) }
}
+ case MasterChanged(masterUrl, masterWebUiUrl) =>
+ logInfo("Master has changed, new master is at " + masterUrl)
+ context.unwatch(master)
+ changeMaster(masterUrl, masterWebUiUrl)
+
+ val execs = executors.values.
+ map(e => new ExecutorDescription(e.appId, e.execId, e.cores, e.state))
+ sender ! WorkerSchedulerStateResponse(workerId, execs.toList)
+
case RegisterWorkerFailed(message) =>
- logError("Worker registration failed: " + message)
- System.exit(1)
-
- case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
- logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
- val manager = new ExecutorRunner(
- appId, execId, appDesc, cores_, memory_, self, workerId, host, new File(execSparkHome_), workDir)
- executors(appId + "/" + execId) = manager
- manager.start()
- coresUsed += cores_
- memoryUsed += memory_
- master ! ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None)
+ if (!registered) {
+ logError("Worker registration failed: " + message)
+ System.exit(1)
+ }
+
+ case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
+ if (masterUrl != activeMasterUrl) {
+ logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.")
+ } else {
+ logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
+ val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
+ self, workerId, host, new File(execSparkHome_), workDir, ExecutorState.RUNNING)
+ executors(appId + "/" + execId) = manager
+ manager.start()
+ coresUsed += cores_
+ memoryUsed += memory_
+ masterLock.synchronized {
+ master ! ExecutorStateChanged(appId, execId, manager.state, None, None)
+ }
+ }
case ExecutorStateChanged(appId, execId, state, message, exitStatus) =>
- master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+ masterLock.synchronized {
+ master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+ }
val fullId = appId + "/" + execId
if (ExecutorState.isFinished(state)) {
val executor = executors(fullId)
@@ -153,32 +220,39 @@ private[spark] class Worker(
memoryUsed -= executor.memory
}
- case KillExecutor(appId, execId) =>
- val fullId = appId + "/" + execId
- executors.get(fullId) match {
- case Some(executor) =>
- logInfo("Asked to kill executor " + fullId)
- executor.kill()
- case None =>
- logInfo("Asked to kill unknown executor " + fullId)
+ case KillExecutor(masterUrl, appId, execId) =>
+ if (masterUrl != activeMasterUrl) {
+ logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor " + execId)
+ } else {
+ val fullId = appId + "/" + execId
+ executors.get(fullId) match {
+ case Some(executor) =>
+ logInfo("Asked to kill executor " + fullId)
+ executor.kill()
+ case None =>
+ logInfo("Asked to kill unknown executor " + fullId)
+ }
}
- case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
+ case Terminated(actor_) if actor_ == master =>
+ masterDisconnected()
+
+ case RemoteClientDisconnected(transport, address) if address == master.path.address =>
+ masterDisconnected()
+
+ case RemoteClientShutdown(transport, address) if address == master.path.address =>
masterDisconnected()
case RequestWorkerState => {
sender ! WorkerStateResponse(host, port, workerId, executors.values.toList,
- finishedExecutors.values.toList, masterUrl, cores, memory,
- coresUsed, memoryUsed, masterWebUiUrl)
+ finishedExecutors.values.toList, activeMasterUrl, cores, memory,
+ coresUsed, memoryUsed, activeMasterWebUiUrl)
}
}
def masterDisconnected() {
- // TODO: It would be nice to try to reconnect to the master, but just shut down for now.
- // (Note that if reconnecting we would also need to assign IDs differently.)
- logError("Connection to master failed! Shutting down.")
- executors.values.foreach(_.kill())
- System.exit(1)
+ logError("Connection to master failed! Waiting for master to reconnect...")
+ connected = false
}
def generateWorkerId(): String = {
@@ -196,17 +270,18 @@ private[spark] object Worker {
def main(argStrings: Array[String]) {
val args = new WorkerArguments(argStrings)
val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores,
- args.memory, args.master, args.workDir)
+ args.memory, args.masters, args.workDir)
actorSystem.awaitTermination()
}
def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
- masterUrl: String, workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = {
+ masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None)
+ : (ActorSystem, Int) = {
// 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")
+ masterUrls, workDir)), name = "Worker")
(actorSystem, boundPort)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index 0ae89a864f..3ed528e6b3 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -29,7 +29,7 @@ private[spark] class WorkerArguments(args: Array[String]) {
var webUiPort = 8081
var cores = inferDefaultCores()
var memory = inferDefaultMemory()
- var master: String = null
+ var masters: Array[String] = null
var workDir: String = null
// Check for settings in environment variables
@@ -86,14 +86,14 @@ private[spark] class WorkerArguments(args: Array[String]) {
printUsageAndExit(0)
case value :: tail =>
- if (master != null) { // Two positional arguments were given
+ if (masters != null) { // Two positional arguments were given
printUsageAndExit(1)
}
- master = value
+ masters = value.stripPrefix("spark://").split(",").map("spark://" + _)
parse(tail)
case Nil =>
- if (master == null) { // No positional argument was given
+ if (masters == null) { // No positional argument was given
printUsageAndExit(1)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
index df269fd047..b7ddd8c816 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala
@@ -25,27 +25,27 @@ private[spark] class WorkerSource(val worker: Worker) extends Source {
val sourceName = "worker"
val metricRegistry = new MetricRegistry()
- metricRegistry.register(MetricRegistry.name("executors", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("executors"), new Gauge[Int] {
override def getValue: Int = worker.executors.size
})
// Gauge for cores used of this worker
- metricRegistry.register(MetricRegistry.name("coresUsed", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("coresUsed"), new Gauge[Int] {
override def getValue: Int = worker.coresUsed
})
// Gauge for memory used of this worker
- metricRegistry.register(MetricRegistry.name("memUsed", "MBytes"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("memUsed_MB"), new Gauge[Int] {
override def getValue: Int = worker.memoryUsed
})
// Gauge for cores free of this worker
- metricRegistry.register(MetricRegistry.name("coresFree", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("coresFree"), new Gauge[Int] {
override def getValue: Int = worker.coresFree
})
// Gauge for memory free of this worker
- metricRegistry.register(MetricRegistry.name("memFree", "MBytes"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("memFree_MB"), new Gauge[Int] {
override def getValue: Int = worker.memoryFree
})
}
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 95d6007f3b..800f1cafcc 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
@@ -105,7 +105,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
val logText = <node>{Utils.offsetBytes(path, startByte, endByte)}</node>
- val linkToMaster = <p><a href={worker.masterWebUiUrl}>Back to Master</a></p>
+ val linkToMaster = <p><a href={worker.activeMasterWebUiUrl}>Back to Master</a></p>
val range = <span>Bytes {startByte.toString} - {endByte.toString} of {logLength}</span>
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 d365804994..20323ea038 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -17,7 +17,7 @@
package org.apache.spark.executor
-import java.io.{File}
+import java.io.File
import java.lang.management.ManagementFactory
import java.nio.ByteBuffer
import java.util.concurrent._
@@ -27,16 +27,17 @@ import scala.collection.mutable.HashMap
import org.apache.spark.scheduler._
import org.apache.spark._
+import org.apache.spark.storage.{StorageLevel, TaskResultBlockId}
import org.apache.spark.util.Utils
-
/**
- * The Mesos executor for Spark.
+ * Spark executor used with Mesos and the standalone scheduler.
*/
private[spark] class Executor(
executorId: String,
slaveHostname: String,
- properties: Seq[(String, String)])
+ properties: Seq[(String, String)],
+ isLocal: Boolean = false)
extends Logging
{
// Application dependencies (added through SparkContext) that we've fetched so far on this node.
@@ -98,21 +99,50 @@ private[spark] class Executor(
}
)
- val executorSource = new ExecutorSource(this)
+ val executorSource = new ExecutorSource(this, executorId)
// Initialize Spark environment (using system properties read above)
- val env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0, false, false)
- SparkEnv.set(env)
- env.metricsSystem.registerSource(executorSource)
+ private val env = {
+ if (!isLocal) {
+ val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0,
+ isDriver = false, isLocal = false)
+ SparkEnv.set(_env)
+ _env.metricsSystem.registerSource(executorSource)
+ _env
+ } else {
+ SparkEnv.get
+ }
+ }
- private val akkaFrameSize = env.actorSystem.settings.config.getBytes("akka.remote.netty.message-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.message-frame-size")
+ }
// Start worker thread pool
val threadPool = new ThreadPoolExecutor(
- 1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable])
+ 1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable], Utils.daemonThreadFactory)
+
+ // Maintains the list of running tasks.
+ private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]
def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) {
- threadPool.execute(new TaskRunner(context, taskId, serializedTask))
+ val tr = new TaskRunner(context, taskId, serializedTask)
+ runningTasks.put(taskId, tr)
+ threadPool.execute(tr)
+ }
+
+ def killTask(taskId: Long) {
+ val tr = runningTasks.get(taskId)
+ if (tr != null) {
+ tr.kill()
+ // We remove the task also in the finally block in TaskRunner.run.
+ // The reason we need to remove it here is because killTask might be called before the task
+ // is even launched, and never reaching that finally block. ConcurrentHashMap's remove is
+ // idempotent.
+ runningTasks.remove(taskId)
+ }
}
/** Get the Yarn approved local directories. */
@@ -124,62 +154,102 @@ private[spark] class Executor(
.getOrElse(Option(System.getenv("LOCAL_DIRS"))
.getOrElse(""))
- if (localDirs.isEmpty()) {
+ if (localDirs.isEmpty) {
throw new Exception("Yarn Local dirs can't be empty")
}
- return localDirs
+ localDirs
}
- class TaskRunner(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer)
+ class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer)
extends Runnable {
+ @volatile private var killed = false
+ @volatile private var task: Task[Any] = _
+
+ def kill() {
+ logInfo("Executor is trying to kill task " + taskId)
+ killed = true
+ if (task != null) {
+ task.kill()
+ }
+ }
+
override def run() {
val startTime = System.currentTimeMillis()
SparkEnv.set(env)
Thread.currentThread.setContextClassLoader(replClassLoader)
val ser = SparkEnv.get.closureSerializer.newInstance()
logInfo("Running task ID " + taskId)
- context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER)
+ execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER)
var attemptedTask: Option[Task[Any]] = None
var taskStart: Long = 0
- def getTotalGCTime = ManagementFactory.getGarbageCollectorMXBeans.map(g => g.getCollectionTime).sum
- val startGCTime = getTotalGCTime
+ def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum
+ val startGCTime = gcTime
try {
SparkEnv.set(env)
Accumulators.clear()
val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask)
updateDependencies(taskFiles, taskJars)
- val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
+ task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
+
+ // If this task has been killed before we deserialized it, let's quit now. Otherwise,
+ // continue executing the task.
+ if (killed) {
+ logInfo("Executor killed task " + taskId)
+ execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
+ return
+ }
+
attemptedTask = Some(task)
- logInfo("Its epoch is " + task.epoch)
+ logDebug("Task " + taskId +"'s epoch is " + task.epoch)
env.mapOutputTracker.updateEpoch(task.epoch)
+
+ // Run the actual task and measure its runtime.
taskStart = System.currentTimeMillis()
val value = task.run(taskId.toInt)
val taskFinish = System.currentTimeMillis()
+
+ // If the task has been killed, let's fail it.
+ if (task.killed) {
+ logInfo("Executor killed task " + taskId)
+ execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
+ return
+ }
+
for (m <- task.metrics) {
- m.hostname = Utils.localHostName
+ m.hostname = Utils.localHostName()
m.executorDeserializeTime = (taskStart - startTime).toInt
m.executorRunTime = (taskFinish - taskStart).toInt
- m.jvmGCTime = getTotalGCTime - startGCTime
+ m.jvmGCTime = gcTime - startGCTime
}
- //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
+ // 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 result = new TaskResult(value, accumUpdates, task.metrics.getOrElse(null))
- val serializedResult = ser.serialize(result)
- logInfo("Serialized size of result for " + taskId + " is " + serializedResult.limit)
- if (serializedResult.limit >= (akkaFrameSize - 1024)) {
- context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(TaskResultTooBigFailure()))
- return
+
+ val directResult = new DirectTaskResult(value, accumUpdates, task.metrics.getOrElse(null))
+ val serializedDirectResult = ser.serialize(directResult)
+ logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit)
+ val serializedResult = {
+ if (serializedDirectResult.limit >= akkaFrameSize - 1024) {
+ logInfo("Storing result for " + taskId + " in local BlockManager")
+ val blockId = TaskResultBlockId(taskId)
+ env.blockManager.putBytes(
+ blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER)
+ ser.serialize(new IndirectTaskResult[Any](blockId))
+ } else {
+ logInfo("Sending result for " + taskId + " directly to driver")
+ serializedDirectResult
+ }
}
- context.statusUpdate(taskId, TaskState.FINISHED, serializedResult)
+
+ execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult)
logInfo("Finished task ID " + taskId)
} catch {
case ffe: FetchFailedException => {
val reason = ffe.toTaskEndReason
- context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
+ execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
}
case t: Throwable => {
@@ -187,10 +257,10 @@ private[spark] class Executor(
val metrics = attemptedTask.flatMap(t => t.metrics)
for (m <- metrics) {
m.executorRunTime = serviceTime
- m.jvmGCTime = getTotalGCTime - startGCTime
+ m.jvmGCTime = gcTime - startGCTime
}
val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics)
- context.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
+ execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
// TODO: Should we exit the whole executor here? On the one hand, the failed task may
// have left some weird state around depending on when the exception was thrown, but on
@@ -198,6 +268,8 @@ private[spark] class Executor(
logError("Exception in task ID " + taskId, t)
//System.exit(1)
}
+ } finally {
+ runningTasks.remove(taskId)
}
}
}
@@ -207,7 +279,7 @@ private[spark] class Executor(
* created by the interpreter to the search path
*/
private def createClassLoader(): ExecutorURLClassLoader = {
- var loader = this.getClass.getClassLoader
+ val loader = this.getClass.getClassLoader
// For each of the jars in the jarSet, add them to the class loader.
// We assume each of the files has already been fetched.
@@ -229,7 +301,7 @@ private[spark] class Executor(
val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader")
.asInstanceOf[Class[_ <: ClassLoader]]
val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader])
- return constructor.newInstance(classUri, parent)
+ constructor.newInstance(classUri, parent)
} catch {
case _: ClassNotFoundException =>
logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!")
@@ -237,7 +309,7 @@ private[spark] class Executor(
null
}
} else {
- return parent
+ parent
}
}
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
index bf8fb4fd21..34ed9c8f73 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
@@ -27,7 +27,7 @@ import scala.collection.JavaConversions._
import org.apache.spark.metrics.source.Source
-class ExecutorSource(val executor: Executor) extends Source {
+class ExecutorSource(val executor: Executor, executorId: String) extends Source {
private def fileStats(scheme: String) : Option[FileSystem.Statistics] =
FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption
@@ -39,34 +39,35 @@ class ExecutorSource(val executor: Executor) extends Source {
}
val metricRegistry = new MetricRegistry()
- val sourceName = "executor"
+ // TODO: It would be nice to pass the application name here
+ val sourceName = "executor.%s".format(executorId)
// Gauge for executor thread pool's actively executing task counts
- metricRegistry.register(MetricRegistry.name("threadpool", "activeTask", "count"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] {
override def getValue: Int = executor.threadPool.getActiveCount()
})
// Gauge for executor thread pool's approximate total number of tasks that have been completed
- metricRegistry.register(MetricRegistry.name("threadpool", "completeTask", "count"), new Gauge[Long] {
+ metricRegistry.register(MetricRegistry.name("threadpool", "completeTasks"), new Gauge[Long] {
override def getValue: Long = executor.threadPool.getCompletedTaskCount()
})
// Gauge for executor thread pool's current number of threads
- metricRegistry.register(MetricRegistry.name("threadpool", "currentPool", "size"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("threadpool", "currentPool_size"), new Gauge[Int] {
override def getValue: Int = executor.threadPool.getPoolSize()
})
// Gauge got executor thread pool's largest number of threads that have ever simultaneously been in th pool
- metricRegistry.register(MetricRegistry.name("threadpool", "maxPool", "size"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("threadpool", "maxPool_size"), new Gauge[Int] {
override def getValue: Int = executor.threadPool.getMaximumPoolSize()
})
// Gauge for file system stats of this executor
for (scheme <- Array("hdfs", "file")) {
- registerFileSystemStat(scheme, "bytesRead", _.getBytesRead(), 0L)
- registerFileSystemStat(scheme, "bytesWritten", _.getBytesWritten(), 0L)
- registerFileSystemStat(scheme, "readOps", _.getReadOps(), 0)
- registerFileSystemStat(scheme, "largeReadOps", _.getLargeReadOps(), 0)
- registerFileSystemStat(scheme, "writeOps", _.getWriteOps(), 0)
+ registerFileSystemStat(scheme, "read_bytes", _.getBytesRead(), 0L)
+ registerFileSystemStat(scheme, "write_bytes", _.getBytesWritten(), 0L)
+ registerFileSystemStat(scheme, "read_ops", _.getReadOps(), 0)
+ registerFileSystemStat(scheme, "largeRead_ops", _.getLargeReadOps(), 0)
+ registerFileSystemStat(scheme, "write_ops", _.getWriteOps(), 0)
}
}
diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
index da62091980..b56d8c9912 100644
--- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala
@@ -18,14 +18,18 @@
package org.apache.spark.executor
import java.nio.ByteBuffer
-import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNativeLibrary, ExecutorDriver}
-import org.apache.mesos.Protos.{TaskState => MesosTaskState, TaskStatus => MesosTaskStatus, _}
-import org.apache.spark.TaskState.TaskState
+
import com.google.protobuf.ByteString
-import org.apache.spark.{Logging}
+
+import org.apache.mesos.{Executor => MesosExecutor, MesosExecutorDriver, MesosNativeLibrary, ExecutorDriver}
+import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _}
+
+import org.apache.spark.Logging
import org.apache.spark.TaskState
+import org.apache.spark.TaskState.TaskState
import org.apache.spark.util.Utils
+
private[spark] class MesosExecutorBackend
extends MesosExecutor
with ExecutorBackend
@@ -71,7 +75,11 @@ private[spark] class MesosExecutorBackend
}
override def killTask(d: ExecutorDriver, t: TaskID) {
- logWarning("Mesos asked us to kill task " + t.getValue + "; ignoring (not yet implemented)")
+ if (executor == null) {
+ logError("Received KillTask but executor was null")
+ } else {
+ executor.killTask(t.getValue.toLong)
+ }
}
override def reregistered(d: ExecutorDriver, p2: SlaveInfo) {}
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..db0bea0472 100644
--- a/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/StandaloneExecutorBackend.scala
@@ -63,12 +63,20 @@ private[spark] class StandaloneExecutorBackend(
case LaunchTask(taskDesc) =>
logInfo("Got assigned task " + taskDesc.taskId)
if (executor == null) {
- logError("Received launchTask but executor was null")
+ logError("Received LaunchTask command but executor was null")
System.exit(1)
} else {
executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask)
}
+ case KillTask(taskId, _) =>
+ if (executor == null) {
+ logError("Received KillTask command but executor was null")
+ System.exit(1)
+ } else {
+ executor.killTask(taskId)
+ }
+
case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
logError("Driver terminated or disconnected! Shutting down.")
System.exit(1)
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
index 0f9c4e00b1..caab748d60 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
@@ -37,10 +37,9 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi
private def setDefaultProperties(prop: Properties) {
prop.setProperty("*.sink.servlet.class", "org.apache.spark.metrics.sink.MetricsServlet")
- prop.setProperty("*.sink.servlet.uri", "/metrics/json")
- prop.setProperty("*.sink.servlet.sample", "false")
- prop.setProperty("master.sink.servlet.uri", "/metrics/master/json")
- prop.setProperty("applications.sink.servlet.uri", "/metrics/applications/json")
+ prop.setProperty("*.sink.servlet.path", "/metrics/json")
+ prop.setProperty("master.sink.servlet.path", "/metrics/master/json")
+ prop.setProperty("applications.sink.servlet.path", "/metrics/applications/json")
}
def initialize() {
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
new file mode 100644
index 0000000000..b924907070
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.metrics.sink
+
+import java.util.Properties
+import java.util.concurrent.TimeUnit
+
+import com.codahale.metrics.ganglia.GangliaReporter
+import com.codahale.metrics.MetricRegistry
+import info.ganglia.gmetric4j.gmetric.GMetric
+
+import org.apache.spark.metrics.MetricsSystem
+
+class GangliaSink(val property: Properties, val registry: MetricRegistry) extends Sink {
+ val GANGLIA_KEY_PERIOD = "period"
+ val GANGLIA_DEFAULT_PERIOD = 10
+
+ val GANGLIA_KEY_UNIT = "unit"
+ val GANGLIA_DEFAULT_UNIT = TimeUnit.SECONDS
+
+ val GANGLIA_KEY_MODE = "mode"
+ val GANGLIA_DEFAULT_MODE = GMetric.UDPAddressingMode.MULTICAST
+
+ // TTL for multicast messages. If listeners are X hops away in network, must be at least X.
+ val GANGLIA_KEY_TTL = "ttl"
+ val GANGLIA_DEFAULT_TTL = 1
+
+ val GANGLIA_KEY_HOST = "host"
+ val GANGLIA_KEY_PORT = "port"
+
+ def propertyToOption(prop: String) = Option(property.getProperty(prop))
+
+ if (!propertyToOption(GANGLIA_KEY_HOST).isDefined) {
+ throw new Exception("Ganglia sink requires 'host' property.")
+ }
+
+ if (!propertyToOption(GANGLIA_KEY_PORT).isDefined) {
+ throw new Exception("Ganglia sink requires 'port' property.")
+ }
+
+ val host = propertyToOption(GANGLIA_KEY_HOST).get
+ val port = propertyToOption(GANGLIA_KEY_PORT).get.toInt
+ val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL)
+ val mode = propertyToOption(GANGLIA_KEY_MODE)
+ .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE)
+ val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt)
+ .getOrElse(GANGLIA_DEFAULT_PERIOD)
+ val pollUnit = propertyToOption(GANGLIA_KEY_UNIT).map(u => TimeUnit.valueOf(u.toUpperCase))
+ .getOrElse(GANGLIA_DEFAULT_UNIT)
+
+ MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod)
+
+ val ganglia = new GMetric(host, port, mode, ttl)
+ val reporter: GangliaReporter = GangliaReporter.forRegistry(registry)
+ .convertDurationsTo(TimeUnit.MILLISECONDS)
+ .convertRatesTo(TimeUnit.SECONDS)
+ .build(ganglia)
+
+ override def start() {
+ reporter.start(pollPeriod, pollUnit)
+ }
+
+ override def stop() {
+ reporter.stop()
+ }
+}
+
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
index 4e90dd4323..99357fede6 100644
--- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala
@@ -31,18 +31,21 @@ import org.eclipse.jetty.server.Handler
import org.apache.spark.ui.JettyUtils
class MetricsServlet(val property: Properties, val registry: MetricRegistry) extends Sink {
- val SERVLET_KEY_URI = "uri"
+ val SERVLET_KEY_PATH = "path"
val SERVLET_KEY_SAMPLE = "sample"
- val servletURI = property.getProperty(SERVLET_KEY_URI)
+ val SERVLET_DEFAULT_SAMPLE = false
- val servletShowSample = property.getProperty(SERVLET_KEY_SAMPLE).toBoolean
+ val servletPath = property.getProperty(SERVLET_KEY_PATH)
+
+ val servletShowSample = Option(property.getProperty(SERVLET_KEY_SAMPLE)).map(_.toBoolean)
+ .getOrElse(SERVLET_DEFAULT_SAMPLE)
val mapper = new ObjectMapper().registerModule(
new MetricsModule(TimeUnit.SECONDS, TimeUnit.MILLISECONDS, servletShowSample))
def getHandlers = Array[(String, Handler)](
- (servletURI, JettyUtils.createHandler(request => getMetricsSnapshot(request), "text/json"))
+ (servletPath, JettyUtils.createHandler(request => getMetricsSnapshot(request), "text/json"))
)
def getMetricsSnapshot(request: HttpServletRequest): String = {
diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
index 3c29700920..1b9fa1e53a 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala
@@ -20,17 +20,18 @@ package org.apache.spark.network.netty
import io.netty.buffer._
import org.apache.spark.Logging
+import org.apache.spark.storage.{TestBlockId, BlockId}
private[spark] class FileHeader (
val fileLen: Int,
- val blockId: String) extends Logging {
+ val blockId: BlockId) extends Logging {
lazy val buffer = {
val buf = Unpooled.buffer()
buf.capacity(FileHeader.HEADER_SIZE)
buf.writeInt(fileLen)
- buf.writeInt(blockId.length)
- blockId.foreach((x: Char) => buf.writeByte(x))
+ buf.writeInt(blockId.name.length)
+ blockId.name.foreach((x: Char) => buf.writeByte(x))
//padding the rest of header
if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) {
buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes)
@@ -57,18 +58,15 @@ private[spark] object FileHeader {
for (i <- 1 to idLength) {
idBuilder += buf.readByte().asInstanceOf[Char]
}
- val blockId = idBuilder.toString()
+ val blockId = BlockId(idBuilder.toString())
new FileHeader(length, blockId)
}
-
- def main (args:Array[String]){
-
- val header = new FileHeader(25,"block_0");
- val buf = header.buffer;
- val newheader = FileHeader.create(buf);
- System.out.println("id="+newheader.blockId+",size="+newheader.fileLen)
-
+ def main (args:Array[String]) {
+ val header = new FileHeader(25, TestBlockId("my_block"))
+ val buf = header.buffer
+ val newHeader = FileHeader.create(buf)
+ System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen)
}
}
diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
index 9493ccffd9..481ff8c3e0 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala
@@ -27,12 +27,13 @@ import org.apache.spark.Logging
import org.apache.spark.network.ConnectionManagerId
import scala.collection.JavaConverters._
+import org.apache.spark.storage.BlockId
private[spark] class ShuffleCopier extends Logging {
- def getBlock(host: String, port: Int, blockId: String,
- resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+ def getBlock(host: String, port: Int, blockId: BlockId,
+ resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback)
val connectTimeout = System.getProperty("spark.shuffle.netty.connect.timeout", "60000").toInt
@@ -41,7 +42,7 @@ private[spark] class ShuffleCopier extends Logging {
try {
fc.init()
fc.connect(host, port)
- fc.sendRequest(blockId)
+ fc.sendRequest(blockId.name)
fc.waitForClose()
fc.close()
} catch {
@@ -53,14 +54,14 @@ private[spark] class ShuffleCopier extends Logging {
}
}
- def getBlock(cmId: ConnectionManagerId, blockId: String,
- resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+ def getBlock(cmId: ConnectionManagerId, blockId: BlockId,
+ resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
getBlock(cmId.host, cmId.port, blockId, resultCollectCallback)
}
def getBlocks(cmId: ConnectionManagerId,
- blocks: Seq[(String, Long)],
- resultCollectCallback: (String, Long, ByteBuf) => Unit) {
+ blocks: Seq[(BlockId, Long)],
+ resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) {
for ((blockId, size) <- blocks) {
getBlock(cmId, blockId, resultCollectCallback)
@@ -71,7 +72,7 @@ private[spark] class ShuffleCopier extends Logging {
private[spark] object ShuffleCopier extends Logging {
- private class ShuffleClientHandler(resultCollectCallBack: (String, Long, ByteBuf) => Unit)
+ private class ShuffleClientHandler(resultCollectCallBack: (BlockId, Long, ByteBuf) => Unit)
extends FileClientHandler with Logging {
override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) {
@@ -79,14 +80,14 @@ private[spark] object ShuffleCopier extends Logging {
resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen))
}
- override def handleError(blockId: String) {
+ override def handleError(blockId: BlockId) {
if (!isComplete) {
resultCollectCallBack(blockId, -1, null)
}
}
}
- def echoResultCollectCallBack(blockId: String, size: Long, content: ByteBuf) {
+ def echoResultCollectCallBack(blockId: BlockId, size: Long, content: ByteBuf) {
if (size != -1) {
logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"")
}
@@ -99,7 +100,7 @@ private[spark] object ShuffleCopier extends Logging {
}
val host = args(0)
val port = args(1).toInt
- val file = args(2)
+ val blockId = BlockId(args(2))
val threads = if (args.length > 3) args(3).toInt else 10
val copiers = Executors.newFixedThreadPool(80)
@@ -107,12 +108,12 @@ private[spark] object ShuffleCopier extends Logging {
Executors.callable(new Runnable() {
def run() {
val copier = new ShuffleCopier()
- copier.getBlock(host, port, file, echoResultCollectCallBack)
+ copier.getBlock(host, port, blockId, echoResultCollectCallBack)
}
})
}).asJava
copiers.invokeAll(tasks)
- copiers.shutdown
+ copiers.shutdown()
System.exit(0)
}
}
diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
index 537f225469..1586dff254 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala
@@ -20,6 +20,8 @@ package org.apache.spark.network.netty
import java.io.File
import org.apache.spark.Logging
+import org.apache.spark.util.Utils
+import org.apache.spark.storage.BlockId
private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging {
@@ -52,16 +54,17 @@ private[spark] object ShuffleSender {
val localDirs = args.drop(2).map(new File(_))
val pResovler = new PathResolver {
- override def getAbsolutePath(blockId: String): String = {
- if (!blockId.startsWith("shuffle_")) {
+ override def getAbsolutePath(blockIdString: String): String = {
+ val blockId = BlockId(blockIdString)
+ if (!blockId.isShuffle) {
throw new Exception("Block " + blockId + " is not a shuffle block")
}
// Figure out which local directory it hashes to, and which subdirectory in that
- val hash = math.abs(blockId.hashCode)
+ val hash = Utils.nonNegativeHash(blockId)
val dirId = hash % localDirs.length
val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
val subDir = new File(localDirs(dirId), "%02x".format(subDirId))
- val file = new File(subDir, blockId)
+ val file = new File(subDir, blockId.name)
return file.getAbsolutePath
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
new file mode 100644
index 0000000000..faaf837be0
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -0,0 +1,122 @@
+/*
+ * 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.rdd
+
+import java.util.concurrent.atomic.AtomicLong
+
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.ExecutionContext.Implicits.global
+
+import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
+
+/**
+ * A set of asynchronous RDD actions available through an implicit conversion.
+ * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
+ */
+class AsyncRDDActions[T: ClassManifest](self: RDD[T]) extends Serializable with Logging {
+
+ /**
+ * Returns a future for counting the number of elements in the RDD.
+ */
+ def countAsync(): FutureAction[Long] = {
+ val totalCount = new AtomicLong
+ self.context.submitJob(
+ self,
+ (iter: Iterator[T]) => {
+ var result = 0L
+ while (iter.hasNext) {
+ result += 1L
+ iter.next()
+ }
+ result
+ },
+ Range(0, self.partitions.size),
+ (index: Int, data: Long) => totalCount.addAndGet(data),
+ totalCount.get())
+ }
+
+ /**
+ * Returns a future for retrieving all elements of this RDD.
+ */
+ def collectAsync(): FutureAction[Seq[T]] = {
+ val results = new Array[Array[T]](self.partitions.size)
+ self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size),
+ (index, data) => results(index) = data, results.flatten.toSeq)
+ }
+
+ /**
+ * Returns a future for retrieving the first num elements of the RDD.
+ */
+ def takeAsync(num: Int): FutureAction[Seq[T]] = {
+ val f = new ComplexFutureAction[Seq[T]]
+
+ f.run {
+ val results = new ArrayBuffer[T](num)
+ val totalParts = self.partitions.length
+ var partsScanned = 0
+ while (results.size < num && partsScanned < totalParts) {
+ // The number of partitions to try in this iteration. It is ok for this number to be
+ // greater than totalParts because we actually cap it at totalParts in runJob.
+ var numPartsToTry = 1
+ if (partsScanned > 0) {
+ // If we didn't find any rows after the first iteration, just try all partitions next.
+ // Otherwise, interpolate the number of partitions we need to try, but overestimate it
+ // by 50%.
+ if (results.size == 0) {
+ numPartsToTry = totalParts - 1
+ } else {
+ numPartsToTry = (1.5 * num * partsScanned / results.size).toInt
+ }
+ }
+ numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions
+
+ val left = num - results.size
+ val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts)
+
+ val buf = new Array[Array[T]](p.size)
+ f.runJob(self,
+ (it: Iterator[T]) => it.take(left).toArray,
+ p,
+ (index: Int, data: Array[T]) => buf(index) = data,
+ Unit)
+
+ buf.foreach(results ++= _.take(num - results.size))
+ partsScanned += numPartsToTry
+ }
+ results.toSeq
+ }
+
+ f
+ }
+
+ /**
+ * Applies a function f to all elements of this RDD.
+ */
+ def foreachAsync(f: T => Unit): FutureAction[Unit] = {
+ self.context.submitJob[T, Unit, Unit](self, _.foreach(f), Range(0, self.partitions.size),
+ (index, data) => Unit, Unit)
+ }
+
+ /**
+ * Applies a function f to each partition of this RDD.
+ */
+ def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = {
+ self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size),
+ (index, data) => Unit, Unit)
+ }
+}
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..44ea573a7c 100644
--- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -18,14 +18,14 @@
package org.apache.spark.rdd
import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext}
-import org.apache.spark.storage.BlockManager
+import org.apache.spark.storage.{BlockId, BlockManager}
-private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition {
+private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition {
val index = idx
}
private[spark]
-class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String])
+class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[BlockId])
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/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
index 3311757189..ccaaecb85b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -85,7 +85,7 @@ private[spark] object CheckpointRDD extends Logging {
val outputDir = new Path(path)
val fs = outputDir.getFileSystem(env.hadoop.newConfiguration())
- val finalOutputName = splitIdToFile(ctx.splitId)
+ val finalOutputName = splitIdToFile(ctx.partitionId)
val finalOutputPath = new Path(outputDir, finalOutputName)
val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId)
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 0187256a8e..911a002884 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -18,13 +18,12 @@
package org.apache.spark.rdd
import java.io.{ObjectOutputStream, IOException}
-import java.util.{HashMap => JHashMap}
-import scala.collection.JavaConversions
import scala.collection.mutable.ArrayBuffer
-import org.apache.spark.{Partition, Partitioner, SparkEnv, TaskContext}
+import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext}
import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency}
+import org.apache.spark.util.AppendOnlyMap
private[spark] sealed trait CoGroupSplitDep extends Serializable
@@ -105,17 +104,14 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
val split = s.asInstanceOf[CoGroupPartition]
val numRdds = split.deps.size
// e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs)
- val map = new JHashMap[K, Seq[ArrayBuffer[Any]]]
+ val map = new AppendOnlyMap[K, Seq[ArrayBuffer[Any]]]
- def getSeq(k: K): Seq[ArrayBuffer[Any]] = {
- val seq = map.get(k)
- if (seq != null) {
- seq
- } else {
- val seq = Array.fill(numRdds)(new ArrayBuffer[Any])
- map.put(k, seq)
- seq
- }
+ val update: (Boolean, Seq[ArrayBuffer[Any]]) => Seq[ArrayBuffer[Any]] = (hadVal, oldVal) => {
+ if (hadVal) oldVal else Array.fill(numRdds)(new ArrayBuffer[Any])
+ }
+
+ val getSeq = (k: K) => {
+ map.changeValue(k, update)
}
val ser = SparkEnv.get.serializerManager.get(serializerClass)
@@ -129,12 +125,12 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
case ShuffleCoGroupSplitDep(shuffleId) => {
// Read map outputs of shuffle
val fetcher = SparkEnv.get.shuffleFetcher
- fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context.taskMetrics, ser).foreach {
+ fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser).foreach {
kv => getSeq(kv._1)(depNum) += kv._2
}
}
}
- JavaConversions.mapAsScalaMap(map).iterator
+ new InterruptibleIterator(context, map.iterator)
}
override def clearDependencies() {
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 2cb6734e41..fad042c7ae 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -19,6 +19,7 @@ package org.apache.spark.rdd
import java.io.EOFException
+import org.apache.hadoop.mapred.FileInputFormat
import org.apache.hadoop.mapred.InputFormat
import org.apache.hadoop.mapred.InputSplit
import org.apache.hadoop.mapred.JobConf
@@ -26,7 +27,8 @@ import org.apache.hadoop.mapred.RecordReader
import org.apache.hadoop.mapred.Reporter
import org.apache.hadoop.util.ReflectionUtils
-import org.apache.spark.{Logging, Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext}
+import org.apache.spark._
+import org.apache.spark.broadcast.Broadcast
import org.apache.spark.util.NextIterator
import org.apache.hadoop.conf.{Configuration, Configurable}
@@ -36,7 +38,7 @@ import org.apache.hadoop.conf.{Configuration, Configurable}
*/
private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSplit)
extends Partition {
-
+
val inputSplit = new SerializableWritable[InputSplit](s)
override def hashCode(): Int = (41 * (41 + rddId) + idx).toInt
@@ -45,29 +47,95 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp
}
/**
- * An RDD that reads a Hadoop dataset as specified by a JobConf (e.g. files in HDFS, the local file
- * system, or S3, tables in HBase, etc).
+ * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS,
+ * 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
+ * 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
+ * creates.
+ * @param inputFormatClass Storage format of the data to be read.
+ * @param keyClass Class of the key associated with the inputFormatClass.
+ * @param valueClass Class of the value associated with the inputFormatClass.
+ * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate.
*/
class HadoopRDD[K, V](
sc: SparkContext,
- @transient conf: JobConf,
+ broadcastedConf: Broadcast[SerializableWritable[Configuration]],
+ initLocalJobConfFuncOpt: Option[JobConf => Unit],
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K],
valueClass: Class[V],
minSplits: Int)
extends RDD[(K, V)](sc, Nil) with Logging {
- // A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it
- private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
+ def this(
+ sc: SparkContext,
+ conf: JobConf,
+ inputFormatClass: Class[_ <: InputFormat[K, V]],
+ keyClass: Class[K],
+ valueClass: Class[V],
+ minSplits: Int) = {
+ this(
+ sc,
+ sc.broadcast(new SerializableWritable(conf))
+ .asInstanceOf[Broadcast[SerializableWritable[Configuration]]],
+ None /* initLocalJobConfFuncOpt */,
+ inputFormatClass,
+ keyClass,
+ valueClass,
+ minSplits)
+ }
+
+ protected val jobConfCacheKey = "rdd_%d_job_conf".format(id)
+
+ protected val inputFormatCacheKey = "rdd_%d_input_format".format(id)
+
+ // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads.
+ protected def getJobConf(): JobConf = {
+ val conf: Configuration = broadcastedConf.value.value
+ if (conf.isInstanceOf[JobConf]) {
+ // A user-broadcasted JobConf was provided to the HadoopRDD, so always use it.
+ return conf.asInstanceOf[JobConf]
+ } else if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) {
+ // getJobConf() has been called previously, so there is already a local cache of the JobConf
+ // needed by this RDD.
+ return HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf]
+ } else {
+ // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the
+ // local process. The local cache is accessed through HadoopRDD.putCachedMetadata().
+ // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects.
+ val newJobConf = new JobConf(broadcastedConf.value.value)
+ initLocalJobConfFuncOpt.map(f => f(newJobConf))
+ HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
+ return newJobConf
+ }
+ }
+
+ protected def getInputFormat(conf: JobConf): InputFormat[K, V] = {
+ if (HadoopRDD.containsCachedMetadata(inputFormatCacheKey)) {
+ return HadoopRDD.getCachedMetadata(inputFormatCacheKey).asInstanceOf[InputFormat[K, V]]
+ }
+ // Once an InputFormat for this RDD is created, cache it so that only one reflection call is
+ // done in each local process.
+ val newInputFormat = ReflectionUtils.newInstance(inputFormatClass.asInstanceOf[Class[_]], conf)
+ .asInstanceOf[InputFormat[K, V]]
+ if (newInputFormat.isInstanceOf[Configurable]) {
+ newInputFormat.asInstanceOf[Configurable].setConf(conf)
+ }
+ HadoopRDD.putCachedMetadata(inputFormatCacheKey, newInputFormat)
+ return newInputFormat
+ }
override def getPartitions: Array[Partition] = {
- val env = SparkEnv.get
- env.hadoop.addCredentials(conf)
- val inputFormat = createInputFormat(conf)
+ val jobConf = getJobConf()
+ val inputFormat = getInputFormat(jobConf)
if (inputFormat.isInstanceOf[Configurable]) {
- inputFormat.asInstanceOf[Configurable].setConf(conf)
+ inputFormat.asInstanceOf[Configurable].setConf(jobConf)
}
- val inputSplits = inputFormat.getSplits(conf, minSplits)
+ val inputSplits = inputFormat.getSplits(jobConf, minSplits)
val array = new Array[Partition](inputSplits.size)
for (i <- 0 until inputSplits.size) {
array(i) = new HadoopPartition(id, i, inputSplits(i))
@@ -75,46 +143,41 @@ class HadoopRDD[K, V](
array
}
- def createInputFormat(conf: JobConf): InputFormat[K, V] = {
- ReflectionUtils.newInstance(inputFormatClass.asInstanceOf[Class[_]], conf)
- .asInstanceOf[InputFormat[K, V]]
- }
-
- override def compute(theSplit: Partition, context: TaskContext) = new NextIterator[(K, V)] {
- val split = theSplit.asInstanceOf[HadoopPartition]
- logInfo("Input split: " + split.inputSplit)
- var reader: RecordReader[K, V] = null
-
- val conf = confBroadcast.value.value
- val fmt = createInputFormat(conf)
- if (fmt.isInstanceOf[Configurable]) {
- fmt.asInstanceOf[Configurable].setConf(conf)
- }
- reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL)
-
- // Register an on-task-completion callback to close the input stream.
- context.addOnCompleteCallback{ () => closeIfNeeded() }
-
- val key: K = reader.createKey()
- val value: V = reader.createValue()
-
- override def getNext() = {
- try {
- finished = !reader.next(key, value)
- } catch {
- case eof: EOFException =>
- finished = true
+ override def compute(theSplit: Partition, context: TaskContext) = {
+ val iter = new NextIterator[(K, V)] {
+ val split = theSplit.asInstanceOf[HadoopPartition]
+ logInfo("Input split: " + split.inputSplit)
+ var reader: RecordReader[K, V] = null
+
+ val jobConf = getJobConf()
+ val inputFormat = getInputFormat(jobConf)
+ reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL)
+
+ // Register an on-task-completion callback to close the input stream.
+ context.addOnCompleteCallback{ () => closeIfNeeded() }
+
+ val key: K = reader.createKey()
+ val value: V = reader.createValue()
+
+ override def getNext() = {
+ try {
+ finished = !reader.next(key, value)
+ } catch {
+ case eof: EOFException =>
+ finished = true
+ }
+ (key, value)
}
- (key, value)
- }
- override def close() {
- try {
- reader.close()
- } catch {
- case e: Exception => logWarning("Exception in RecordReader.close()", e)
+ override def close() {
+ try {
+ reader.close()
+ } catch {
+ case e: Exception => logWarning("Exception in RecordReader.close()", e)
+ }
}
}
+ new InterruptibleIterator[(K, V)](context, iter)
}
override def getPreferredLocations(split: Partition): Seq[String] = {
@@ -127,5 +190,18 @@ class HadoopRDD[K, V](
// Do nothing. Hadoop RDD should not be checkpointed.
}
- def getConf: Configuration = confBroadcast.value.value
+ def getConf: Configuration = getJobConf()
+}
+
+private[spark] object HadoopRDD {
+ /**
+ * The three methods below are helpers for accessing the local map, a property of the SparkEnv of
+ * the local process.
+ */
+ def getCachedMetadata(key: String) = SparkEnv.get.hadoop.hadoopJobMetadata.get(key)
+
+ def containsCachedMetadata(key: String) = SparkEnv.get.hadoop.hadoopJobMetadata.containsKey(key)
+
+ def putCachedMetadata(key: String, value: Any) =
+ SparkEnv.get.hadoop.hadoopJobMetadata.put(key, value)
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala
index 3ed8339010..aea08ff81b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithIndexRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala
@@ -21,14 +21,14 @@ import org.apache.spark.{Partition, TaskContext}
/**
- * A variant of the MapPartitionsRDD that passes the partition index into the
- * closure. This can be used to generate or collect partition specific
- * information such as the number of tuples in a partition.
+ * A variant of the MapPartitionsRDD that passes the TaskContext into the closure. From the
+ * TaskContext, the closure can either get access to the interruptible flag or get the index
+ * of the partition in the RDD.
*/
private[spark]
-class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest](
+class MapPartitionsWithContextRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
- f: (Int, Iterator[T]) => Iterator[U],
+ f: (TaskContext, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean
) extends RDD[U](prev) {
@@ -37,5 +37,5 @@ class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest](
override val partitioner = if (preservesPartitioning) prev.partitioner else None
override def compute(split: Partition, context: TaskContext) =
- f(split.index, firstParent[T].iterator(split, context))
+ f(context, firstParent[T].iterator(split, context))
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index 7b3a89f7e0..2662d48c84 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapreduce._
-import org.apache.spark.{Dependency, Logging, Partition, SerializableWritable, SparkContext, TaskContext}
+import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext}
private[spark]
@@ -71,49 +71,52 @@ class NewHadoopRDD[K, V](
result
}
- override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] {
- val split = theSplit.asInstanceOf[NewHadoopPartition]
- logInfo("Input split: " + split.serializableHadoopSplit)
- val conf = confBroadcast.value.value
- val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0)
- val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId)
- val format = inputFormatClass.newInstance
- if (format.isInstanceOf[Configurable]) {
- format.asInstanceOf[Configurable].setConf(conf)
- }
- val reader = format.createRecordReader(
- split.serializableHadoopSplit.value, hadoopAttemptContext)
- reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
-
- // Register an on-task-completion callback to close the input stream.
- context.addOnCompleteCallback(() => close())
-
- var havePair = false
- var finished = false
-
- override def hasNext: Boolean = {
- if (!finished && !havePair) {
- finished = !reader.nextKeyValue
- havePair = !finished
+ override def compute(theSplit: Partition, context: TaskContext) = {
+ val iter = new Iterator[(K, V)] {
+ val split = theSplit.asInstanceOf[NewHadoopPartition]
+ logInfo("Input split: " + split.serializableHadoopSplit)
+ val conf = confBroadcast.value.value
+ val attemptId = newTaskAttemptID(jobtrackerId, id, true, split.index, 0)
+ val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId)
+ val format = inputFormatClass.newInstance
+ if (format.isInstanceOf[Configurable]) {
+ format.asInstanceOf[Configurable].setConf(conf)
+ }
+ val reader = format.createRecordReader(
+ split.serializableHadoopSplit.value, hadoopAttemptContext)
+ reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
+
+ // Register an on-task-completion callback to close the input stream.
+ context.addOnCompleteCallback(() => close())
+
+ var havePair = false
+ var finished = false
+
+ override def hasNext: Boolean = {
+ if (!finished && !havePair) {
+ finished = !reader.nextKeyValue
+ havePair = !finished
+ }
+ !finished
}
- !finished
- }
- override def next: (K, V) = {
- if (!hasNext) {
- throw new java.util.NoSuchElementException("End of stream")
+ override def next(): (K, V) = {
+ if (!hasNext) {
+ throw new java.util.NoSuchElementException("End of stream")
+ }
+ havePair = false
+ (reader.getCurrentKey, reader.getCurrentValue)
}
- havePair = false
- return (reader.getCurrentKey, reader.getCurrentValue)
- }
- private def close() {
- try {
- reader.close()
- } catch {
- case e: Exception => logWarning("Exception in RecordReader.close()", e)
+ private def close() {
+ try {
+ reader.close()
+ } catch {
+ case e: Exception => logWarning("Exception in RecordReader.close()", e)
+ }
}
}
+ new InterruptibleIterator(context, iter)
}
override def getPreferredLocations(split: Partition): Seq[String] = {
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 a47c512275..93b78e1232 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -84,18 +84,24 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
}
val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
if (self.partitioner == Some(partitioner)) {
- self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
+ self.mapPartitionsWithContext((context, iter) => {
+ new InterruptibleIterator(context, aggregator.combineValuesByKey(iter))
+ }, preservesPartitioning = true)
} else if (mapSideCombine) {
val combined = self.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner)
.setSerializer(serializerClass)
- partitioned.mapPartitions(aggregator.combineCombinersByKey, preservesPartitioning = true)
+ partitioned.mapPartitionsWithContext((context, iter) => {
+ new InterruptibleIterator(context, aggregator.combineCombinersByKey(iter))
+ }, preservesPartitioning = true)
} else {
// Don't apply map-side combiner.
// A sanity check to make sure mergeCombiners is not defined.
assert(mergeCombiners == null)
val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass)
- values.mapPartitions(aggregator.combineValuesByKey, preservesPartitioning = true)
+ values.mapPartitionsWithContext((context, iter) => {
+ new InterruptibleIterator(context, aggregator.combineValuesByKey(iter))
+ }, preservesPartitioning = true)
}
}
@@ -564,7 +570,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
// around by taking a mod. We expect that no task will be attempted 2 billion times.
val attemptNumber = (context.attemptId % Int.MaxValue).toInt
/* "reduce task" <split #> <attempt # = spark task #> */
- val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.splitId, attemptNumber)
+ val attemptId = newTaskAttemptID(jobtrackerID, stageId, false, context.partitionId, attemptNumber)
val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
val format = outputFormatClass.newInstance
val committer = format.getOutputCommitter(hadoopContext)
@@ -663,7 +669,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
// around by taking a mod. We expect that no task will be attempted 2 billion times.
val attemptNumber = (context.attemptId % Int.MaxValue).toInt
- writer.setup(context.stageId, context.splitId, attemptNumber)
+ writer.setup(context.stageId, context.partitionId, attemptNumber)
writer.open()
var count = 0
diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
index 6dbd4309aa..cd96250389 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
@@ -94,8 +94,9 @@ private[spark] class ParallelCollectionRDD[T: ClassManifest](
slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray
}
- override def compute(s: Partition, context: TaskContext) =
- s.asInstanceOf[ParallelCollectionPartition[T]].iterator
+ override def compute(s: Partition, context: TaskContext) = {
+ new InterruptibleIterator(context, s.asInstanceOf[ParallelCollectionPartition[T]].iterator)
+ }
override def getPreferredLocations(s: Partition): Seq[String] = {
locationPrefs.getOrElse(s.index, Nil)
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 e143ecd096..0355618e43 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -267,6 +267,23 @@ abstract class RDD[T: ClassManifest](
/**
* Return a new RDD that is reduced into `numPartitions` partitions.
+ *
+ * This results in a narrow dependency, e.g. if you go from 1000 partitions
+ * to 100 partitions, there will not be a shuffle, instead each of the 100
+ * new partitions will claim 10 of the current partitions.
+ *
+ * However, if you're doing a drastic coalesce, e.g. to numPartitions = 1,
+ * this may result in your computation taking place on fewer nodes than
+ * you like (e.g. one node in the case of numPartitions = 1). To avoid this,
+ * you can pass shuffle = true. This will add a shuffle step, but means the
+ * current upstream partitions will be executed in parallel (per whatever
+ * the current partitioning is).
+ *
+ * Note: With shuffle = true, you can actually coalesce to a larger number
+ * of partitions. This is useful if you have a small number of partitions,
+ * say 100, potentially with a few partitions being abnormally large. Calling
+ * coalesce(1000, shuffle = true) will result in 1000 partitions with the
+ * data distributed using a hash partitioner.
*/
def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = {
if (shuffle) {
@@ -401,26 +418,39 @@ abstract class RDD[T: ClassManifest](
command: Seq[String],
env: Map[String, String] = Map(),
printPipeContext: (String => Unit) => Unit = null,
- printRDDElement: (T, String => Unit) => Unit = null): RDD[String] =
+ printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = {
new PipedRDD(this, command, env,
if (printPipeContext ne null) sc.clean(printPipeContext) else null,
if (printRDDElement ne null) sc.clean(printRDDElement) else null)
+ }
/**
* Return a new RDD by applying a function to each partition of this RDD.
*/
- def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U],
- preservesPartitioning: Boolean = false): RDD[U] =
+ def mapPartitions[U: ClassManifest](
+ f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning)
+ }
/**
* Return a new RDD by applying a function to each partition of this RDD, while tracking the index
* of the original partition.
*/
def mapPartitionsWithIndex[U: ClassManifest](
- f: (Int, Iterator[T]) => Iterator[U],
- preservesPartitioning: Boolean = false): RDD[U] =
- new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
+ f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
+ val func = (context: TaskContext, iter: Iterator[T]) => f(context.partitionId, iter)
+ new MapPartitionsWithContextRDD(this, sc.clean(func), preservesPartitioning)
+ }
+
+ /**
+ * Return a new RDD by applying a function to each partition of this RDD. This is a variant of
+ * mapPartitions that also passes the TaskContext into the closure.
+ */
+ def mapPartitionsWithContext[U: ClassManifest](
+ f: (TaskContext, Iterator[T]) => Iterator[U],
+ preservesPartitioning: Boolean = false): RDD[U] = {
+ new MapPartitionsWithContextRDD(this, sc.clean(f), preservesPartitioning)
+ }
/**
* Return a new RDD by applying a function to each partition of this RDD, while tracking the index
@@ -428,22 +458,23 @@ abstract class RDD[T: ClassManifest](
*/
@deprecated("use mapPartitionsWithIndex", "0.7.0")
def mapPartitionsWithSplit[U: ClassManifest](
- f: (Int, Iterator[T]) => Iterator[U],
- preservesPartitioning: Boolean = false): RDD[U] =
- new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
+ f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
+ mapPartitionsWithIndex(f, preservesPartitioning)
+ }
/**
* Maps f over this RDD, where f takes an additional parameter of type A. This
* additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def mapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false)
- (f:(T, A) => U): RDD[U] = {
- def iterF(index: Int, iter: Iterator[T]): Iterator[U] = {
- val a = constructA(index)
- iter.map(t => f(t, a))
- }
- new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning)
+ def mapWith[A: ClassManifest, U: ClassManifest]
+ (constructA: Int => A, preservesPartitioning: Boolean = false)
+ (f: (T, A) => U): RDD[U] = {
+ def iterF(context: TaskContext, iter: Iterator[T]): Iterator[U] = {
+ val a = constructA(context.partitionId)
+ iter.map(t => f(t, a))
+ }
+ new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning)
}
/**
@@ -451,13 +482,14 @@ abstract class RDD[T: ClassManifest](
* additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def flatMapWith[A: ClassManifest, U: ClassManifest](constructA: Int => A, preservesPartitioning: Boolean = false)
- (f:(T, A) => Seq[U]): RDD[U] = {
- def iterF(index: Int, iter: Iterator[T]): Iterator[U] = {
- val a = constructA(index)
- iter.flatMap(t => f(t, a))
- }
- new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), preservesPartitioning)
+ def flatMapWith[A: ClassManifest, U: ClassManifest]
+ (constructA: Int => A, preservesPartitioning: Boolean = false)
+ (f: (T, A) => Seq[U]): RDD[U] = {
+ def iterF(context: TaskContext, iter: Iterator[T]): Iterator[U] = {
+ val a = constructA(context.partitionId)
+ iter.flatMap(t => f(t, a))
+ }
+ new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning)
}
/**
@@ -465,13 +497,12 @@ abstract class RDD[T: ClassManifest](
* This additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def foreachWith[A: ClassManifest](constructA: Int => A)
- (f:(T, A) => Unit) {
- def iterF(index: Int, iter: Iterator[T]): Iterator[T] = {
- val a = constructA(index)
- iter.map(t => {f(t, a); t})
- }
- (new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true)).foreach(_ => {})
+ def foreachWith[A: ClassManifest](constructA: Int => A)(f: (T, A) => Unit) {
+ def iterF(context: TaskContext, iter: Iterator[T]): Iterator[T] = {
+ val a = constructA(context.partitionId)
+ iter.map(t => {f(t, a); t})
+ }
+ new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true).foreach(_ => {})
}
/**
@@ -479,13 +510,12 @@ abstract class RDD[T: ClassManifest](
* additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def filterWith[A: ClassManifest](constructA: Int => A)
- (p:(T, A) => Boolean): RDD[T] = {
- def iterF(index: Int, iter: Iterator[T]): Iterator[T] = {
- val a = constructA(index)
- iter.filter(t => p(t, a))
- }
- new MapPartitionsWithIndexRDD(this, sc.clean(iterF _), true)
+ def filterWith[A: ClassManifest](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = {
+ def iterF(context: TaskContext, iter: Iterator[T]): Iterator[T] = {
+ val a = constructA(context.partitionId)
+ iter.filter(t => p(t, a))
+ }
+ new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true)
}
/**
@@ -524,16 +554,14 @@ abstract class RDD[T: ClassManifest](
* Applies a function f to all elements of this RDD.
*/
def foreach(f: T => Unit) {
- val cleanF = sc.clean(f)
- sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF))
+ sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f))
}
/**
* Applies a function f to each partition of this RDD.
*/
def foreachPartition(f: Iterator[T] => Unit) {
- val cleanF = sc.clean(f)
- sc.runJob(this, (iter: Iterator[T]) => cleanF(iter))
+ sc.runJob(this, (iter: Iterator[T]) => f(iter))
}
/**
@@ -658,6 +686,8 @@ abstract class RDD[T: ClassManifest](
*/
def count(): Long = {
sc.runJob(this, (iter: Iterator[T]) => {
+ // Use a while loop to count the number of elements rather than iter.size because
+ // iter.size uses a for loop, which is slightly slower in current version of Scala.
var result = 0L
while (iter.hasNext) {
result += 1L
@@ -736,24 +766,42 @@ abstract class RDD[T: ClassManifest](
}
/**
- * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so
- * it will be slow if a lot of partitions are required. In that case, use collect() to get the
- * whole RDD instead.
+ * Take the first num elements of the RDD. It works by first scanning one partition, and use the
+ * results from that partition to estimate the number of additional partitions needed to satisfy
+ * the limit.
*/
def take(num: Int): Array[T] = {
if (num == 0) {
return new Array[T](0)
}
+
val buf = new ArrayBuffer[T]
- var p = 0
- while (buf.size < num && p < partitions.size) {
+ val totalParts = this.partitions.length
+ var partsScanned = 0
+ while (buf.size < num && partsScanned < totalParts) {
+ // The number of partitions to try in this iteration. It is ok for this number to be
+ // greater than totalParts because we actually cap it at totalParts in runJob.
+ var numPartsToTry = 1
+ if (partsScanned > 0) {
+ // If we didn't find any rows after the first iteration, just try all partitions next.
+ // Otherwise, interpolate the number of partitions we need to try, but overestimate it
+ // by 50%.
+ if (buf.size == 0) {
+ numPartsToTry = totalParts - 1
+ } else {
+ numPartsToTry = (1.5 * num * partsScanned / buf.size).toInt
+ }
+ }
+ numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions
+
val left = num - buf.size
- val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, Array(p), true)
- buf ++= res(0)
- if (buf.size == num)
- return buf.toArray
- p += 1
+ val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts)
+ val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p, allowLocal = true)
+
+ res.foreach(buf ++= _.take(num - buf.size))
+ partsScanned += numPartsToTry
}
+
return buf.toArray
}
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..a5d751a7bd 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -56,7 +56,7 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassManifest](
override def compute(split: Partition, context: TaskContext): Iterator[P] = {
val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
- SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context.taskMetrics,
+ SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context,
SparkEnv.get.serializerManager.get(serializerClass))
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
index 8c1a29dfff..7af4d803e7 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -108,7 +108,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM
}
case ShuffleCoGroupSplitDep(shuffleId) => {
val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index,
- context.taskMetrics, serializer)
+ context, serializer)
iter.foreach(op)
}
}
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 92add5b073..7fb614402b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -28,9 +28,8 @@ import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
-import org.apache.spark.scheduler.cluster.TaskInfo
-import org.apache.spark.storage.{BlockManager, BlockManagerMaster}
-import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId}
+import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap}
/**
* The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of
@@ -42,11 +41,11 @@ import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
* locations to run each task on, based on the current cache status, and passes these to the
* low-level TaskScheduler. Furthermore, it handles failures due to shuffle output files being
* lost, in which case old stages may need to be resubmitted. Failures *within* a stage that are
- * not caused by shuffie file loss are handled by the TaskScheduler, which will retry each task
+ * not caused by shuffle file loss are handled by the TaskScheduler, which will retry each task
* a small number of times before cancelling the whole stage.
*
* THREADING: This class runs all its logic in a single thread executing the run() method, to which
- * events are submitted using a synchonized queue (eventQueue). The public API methods, such as
+ * events are submitted using a synchronized queue (eventQueue). The public API methods, such as
* runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods
* should be private.
*/
@@ -89,7 +88,8 @@ class DAGScheduler(
eventQueue.put(ExecutorGained(execId, host))
}
- // Called by TaskScheduler to cancel an entire TaskSet due to repeated failures.
+ // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or
+ // cancellation of the job itself.
override def taskSetFailed(taskSet: TaskSet, reason: String) {
eventQueue.put(TaskSetFailed(taskSet, reason))
}
@@ -105,17 +105,19 @@ class DAGScheduler(
private val eventQueue = new LinkedBlockingQueue[DAGSchedulerEvent]
- val nextJobId = new AtomicInteger(0)
+ private[scheduler] val nextJobId = new AtomicInteger(0)
- val nextStageId = new AtomicInteger(0)
+ def numTotalJobs: Int = nextJobId.get()
- val stageIdToStage = new TimeStampedHashMap[Int, Stage]
+ private val nextStageId = new AtomicInteger(0)
- val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
+ private val stageIdToStage = new TimeStampedHashMap[Int, Stage]
+
+ private val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo]
- private val listenerBus = new SparkListenerBus()
+ private[spark] val listenerBus = new SparkListenerBus()
// Contains the locations that each RDD's partitions are cached on
private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]]
@@ -128,6 +130,7 @@ class DAGScheduler(
// stray messages to detect.
val failedEpoch = new HashMap[String, Long]
+ // stage id to the active job
val idToActiveJob = new HashMap[Int, ActiveJob]
val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done
@@ -139,7 +142,7 @@ class DAGScheduler(
val activeJobs = new HashSet[ActiveJob]
val resultStageToJob = new HashMap[Stage, ActiveJob]
- val metadataCleaner = new MetadataCleaner("DAGScheduler", this.cleanup)
+ val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup)
// Start a thread to run the DAGScheduler event loop
def start() {
@@ -157,7 +160,7 @@ class DAGScheduler(
private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = {
if (!cacheLocs.contains(rdd.id)) {
- val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray
+ val blockIds = rdd.partitions.indices.map(index=> RDDBlockId(rdd.id, index)).toArray[BlockId]
val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster)
cacheLocs(rdd.id) = blockIds.map { id =>
locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId))
@@ -262,32 +265,41 @@ class DAGScheduler(
}
/**
- * Returns (and does not submit) a JobSubmitted event suitable to run a given job, and a
- * JobWaiter whose getResult() method will return the result of the job when it is complete.
- *
- * The job is assumed to have at least one partition; zero partition jobs should be handled
- * without a JobSubmitted event.
+ * Submit a job to the job scheduler and get a JobWaiter object back. The JobWaiter object
+ * can be used to block until the the job finishes executing or can be used to cancel the job.
*/
- private[scheduler] def prepareJob[T, U: ClassManifest](
- finalRdd: RDD[T],
+ def submitJob[T, U](
+ rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
callSite: String,
allowLocal: Boolean,
resultHandler: (Int, U) => Unit,
- properties: Properties = null)
- : (JobSubmitted, JobWaiter[U]) =
+ properties: Properties = null): JobWaiter[U] =
{
+ val jobId = nextJobId.getAndIncrement()
+ if (partitions.size == 0) {
+ return new JobWaiter[U](this, jobId, 0, resultHandler)
+ }
+
+ // Check to make sure we are not launching a task on a partition that does not exist.
+ val maxPartitions = rdd.partitions.length
+ partitions.find(p => p >= maxPartitions).foreach { p =>
+ throw new IllegalArgumentException(
+ "Attempting to access a non-existent partition: " + p + ". " +
+ "Total number of partitions: " + maxPartitions)
+ }
+
assert(partitions.size > 0)
- val waiter = new JobWaiter(partitions.size, resultHandler)
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
- val toSubmit = JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter,
- properties)
- (toSubmit, waiter)
+ val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
+ eventQueue.put(JobSubmitted(jobId, rdd, func2, partitions.toArray, allowLocal, callSite,
+ waiter, properties))
+ waiter
}
def runJob[T, U: ClassManifest](
- finalRdd: RDD[T],
+ rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
callSite: String,
@@ -295,21 +307,7 @@ class DAGScheduler(
resultHandler: (Int, U) => Unit,
properties: Properties = null)
{
- if (partitions.size == 0) {
- return
- }
-
- // Check to make sure we are not launching a task on a partition that does not exist.
- val maxPartitions = finalRdd.partitions.length
- partitions.find(p => p >= maxPartitions).foreach { p =>
- throw new IllegalArgumentException(
- "Attempting to access a non-existent partition: " + p + ". " +
- "Total number of partitions: " + maxPartitions)
- }
-
- val (toSubmit: JobSubmitted, waiter: JobWaiter[_]) = prepareJob(
- finalRdd, func, partitions, callSite, allowLocal, resultHandler, properties)
- eventQueue.put(toSubmit)
+ val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties)
waiter.awaitResult() match {
case JobSucceeded => {}
case JobFailed(exception: Exception, _) =>
@@ -330,19 +328,35 @@ class DAGScheduler(
val listener = new ApproximateActionListener(rdd, func, evaluator, timeout)
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
val partitions = (0 until rdd.partitions.size).toArray
- eventQueue.put(JobSubmitted(rdd, func2, partitions, allowLocal = false, callSite, listener, properties))
+ val jobId = nextJobId.getAndIncrement()
+ eventQueue.put(JobSubmitted(jobId, rdd, func2, partitions, allowLocal = false, callSite,
+ listener, properties))
listener.awaitResult() // Will throw an exception if the job fails
}
/**
+ * Cancel a job that is running or waiting in the queue.
+ */
+ def cancelJob(jobId: Int) {
+ logInfo("Asked to cancel job " + jobId)
+ eventQueue.put(JobCancelled(jobId))
+ }
+
+ /**
+ * Cancel all jobs that are running or waiting in the queue.
+ */
+ def cancelAllJobs() {
+ eventQueue.put(AllJobsCancelled)
+ }
+
+ /**
* Process one event retrieved from the event queue.
* Returns true if we should stop the event loop.
*/
private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = {
event match {
- case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener, properties) =>
- val jobId = nextJobId.getAndIncrement()
- val finalStage = newStage(finalRDD, None, jobId, Some(callSite))
+ case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
+ val finalStage = newStage(rdd, None, jobId, Some(callSite))
val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties)
clearCacheLocs()
logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length +
@@ -361,6 +375,18 @@ class DAGScheduler(
submitStage(finalStage)
}
+ case JobCancelled(jobId) =>
+ // Cancel a job: find all the running stages that are linked to this job, and cancel them.
+ running.filter(_.jobId == jobId).foreach { stage =>
+ taskSched.cancelTasks(stage.id)
+ }
+
+ case AllJobsCancelled =>
+ // Cancel all running jobs.
+ running.foreach { stage =>
+ taskSched.cancelTasks(stage.id)
+ }
+
case ExecutorGained(execId, host) =>
handleExecutorGained(execId, host)
@@ -478,7 +504,8 @@ class DAGScheduler(
SparkEnv.set(env)
val rdd = job.finalStage.rdd
val split = rdd.partitions(job.partitions(0))
- val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0)
+ val taskContext =
+ new TaskContext(job.finalStage.id, job.partitions(0), 0, runningLocally = true)
try {
val result = job.func(taskContext, rdd.iterator(split, taskContext))
job.listener.taskSucceeded(0, result)
@@ -531,9 +558,16 @@ class DAGScheduler(
tasks += new ResultTask(stage.id, stage.rdd, job.func, partition, locs, id)
}
}
+
+ val properties = if (idToActiveJob.contains(stage.jobId)) {
+ idToActiveJob(stage.jobId).properties
+ } else {
+ //this stage will be assigned to "default" pool
+ null
+ }
+
// must be run listener before possible NotSerializableException
// should be "StageSubmitted" first and then "JobEnded"
- val properties = idToActiveJob(stage.jobId).properties
listenerBus.post(SparkListenerStageSubmitted(stage, tasks.size, properties))
if (tasks.size > 0) {
@@ -545,7 +579,7 @@ class DAGScheduler(
SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head)
} catch {
case e: NotSerializableException =>
- abortStage(stage, e.toString)
+ abortStage(stage, "Task not serializable: " + e.toString)
running -= stage
return
}
@@ -571,6 +605,11 @@ class DAGScheduler(
*/
private def handleTaskCompletion(event: CompletionEvent) {
val task = event.task
+
+ if (!stageIdToStage.contains(task.stageId)) {
+ // Skip all the actions if the stage has been cancelled.
+ return
+ }
val stage = stageIdToStage(task.stageId)
def markStageAsFinished(stage: Stage) = {
@@ -619,7 +658,7 @@ class DAGScheduler(
if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) {
logInfo("Ignoring possibly bogus ShuffleMapTask completion from " + execId)
} else {
- stage.addOutputLoc(smt.partition, status)
+ stage.addOutputLoc(smt.partitionId, status)
}
if (running.contains(stage) && pendingTasks(stage).isEmpty) {
markStageAsFinished(stage)
@@ -697,6 +736,9 @@ class DAGScheduler(
case ExceptionFailure(className, description, stackTrace, metrics) =>
// Do nothing here, left up to the TaskScheduler to decide how to handle user failures
+ case TaskResultLost =>
+ // Do nothing here; the TaskScheduler handles these failures and resubmits the task.
+
case other =>
// Unrecognized failure - abort all jobs depending on this stage
abortStage(stageIdToStage(task.stageId), task + " failed: " + other)
@@ -742,14 +784,14 @@ class DAGScheduler(
/**
* Aborts all jobs depending on a particular Stage. This is called in response to a task set
- * being cancelled by the TaskScheduler. Use taskSetFailed() to inject this event from outside.
+ * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside.
*/
private def abortStage(failedStage: Stage, reason: String) {
val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq
failedStage.completionTime = Some(System.currentTimeMillis())
for (resultStage <- dependentStages) {
val job = resultStageToJob(resultStage)
- val error = new SparkException("Job failed: " + reason)
+ val error = new SparkException("Job aborted: " + reason)
job.listener.jobFailed(error)
listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))
idToActiveJob -= resultStage.jobId
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
index 0d99670648..ee89bfb38d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
@@ -19,7 +19,6 @@ package org.apache.spark.scheduler
import java.util.Properties
-import org.apache.spark.scheduler.cluster.TaskInfo
import scala.collection.mutable.Map
import org.apache.spark._
@@ -32,9 +31,10 @@ import org.apache.spark.executor.TaskMetrics
* submitted) but there is a single "logic" thread that reads these events and takes decisions.
* This greatly simplifies synchronization.
*/
-private[spark] sealed trait DAGSchedulerEvent
+private[scheduler] sealed trait DAGSchedulerEvent
-private[spark] case class JobSubmitted(
+private[scheduler] case class JobSubmitted(
+ jobId: Int,
finalRDD: RDD[_],
func: (TaskContext, Iterator[_]) => _,
partitions: Array[Int],
@@ -44,9 +44,14 @@ private[spark] case class JobSubmitted(
properties: Properties = null)
extends DAGSchedulerEvent
-private[spark] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent
+private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent
-private[spark] case class CompletionEvent(
+private[scheduler] case object AllJobsCancelled extends DAGSchedulerEvent
+
+private[scheduler]
+case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent
+
+private[scheduler] case class CompletionEvent(
task: Task[_],
reason: TaskEndReason,
result: Any,
@@ -55,10 +60,12 @@ private[spark] case class CompletionEvent(
taskMetrics: TaskMetrics)
extends DAGSchedulerEvent
-private[spark] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent
+private[scheduler]
+case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent
-private[spark] case class ExecutorLost(execId: String) extends DAGSchedulerEvent
+private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent
-private[spark] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent
+private[scheduler]
+case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent
-private[spark] case object StopDAGScheduler extends DAGSchedulerEvent
+private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
index 22e3723ac8..7b5c0e29ad 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
@@ -20,28 +20,30 @@ package org.apache.spark.scheduler
import com.codahale.metrics.{Gauge,MetricRegistry}
import org.apache.spark.metrics.source.Source
+import org.apache.spark.SparkContext
-private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source {
+private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext)
+ extends Source {
val metricRegistry = new MetricRegistry()
- val sourceName = "DAGScheduler"
+ val sourceName = "%s.DAGScheduler".format(sc.appName)
- metricRegistry.register(MetricRegistry.name("stage", "failedStages", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] {
override def getValue: Int = dagScheduler.failed.size
})
- metricRegistry.register(MetricRegistry.name("stage", "runningStages", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("stage", "runningStages"), new Gauge[Int] {
override def getValue: Int = dagScheduler.running.size
})
- metricRegistry.register(MetricRegistry.name("stage", "waitingStages", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("stage", "waitingStages"), new Gauge[Int] {
override def getValue: Int = dagScheduler.waiting.size
})
- metricRegistry.register(MetricRegistry.name("job", "allJobs", "number"), new Gauge[Int] {
- override def getValue: Int = dagScheduler.nextJobId.get()
+ metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] {
+ override def getValue: Int = dagScheduler.numTotalJobs
})
- metricRegistry.register(MetricRegistry.name("job", "activeJobs", "number"), new Gauge[Int] {
+ metricRegistry.register(MetricRegistry.name("job", "activeJobs"), new Gauge[Int] {
override def getValue: Int = dagScheduler.activeJobs.size
})
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
index c8b78bf00a..3628b1b078 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -30,7 +30,6 @@ import scala.io.Source
import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.scheduler.cluster.TaskInfo
// Used to record runtime information for each job, including RDD graph
// tasks' start/stop shuffle information and information from outside
diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
index 200d881799..58f238d8cf 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
@@ -17,48 +17,58 @@
package org.apache.spark.scheduler
-import scala.collection.mutable.ArrayBuffer
-
/**
* An object that waits for a DAGScheduler job to complete. As tasks finish, it passes their
* results to the given handler function.
*/
-private[spark] class JobWaiter[T](totalTasks: Int, resultHandler: (Int, T) => Unit)
+private[spark] class JobWaiter[T](
+ dagScheduler: DAGScheduler,
+ jobId: Int,
+ totalTasks: Int,
+ resultHandler: (Int, T) => Unit)
extends JobListener {
private var finishedTasks = 0
- private var jobFinished = false // Is the job as a whole finished (succeeded or failed)?
- private var jobResult: JobResult = null // If the job is finished, this will be its result
+ // Is the job as a whole finished (succeeded or failed)?
+ private var _jobFinished = totalTasks == 0
- override def taskSucceeded(index: Int, result: Any) {
- synchronized {
- if (jobFinished) {
- throw new UnsupportedOperationException("taskSucceeded() called on a finished JobWaiter")
- }
- resultHandler(index, result.asInstanceOf[T])
- finishedTasks += 1
- if (finishedTasks == totalTasks) {
- jobFinished = true
- jobResult = JobSucceeded
- this.notifyAll()
- }
- }
+ def jobFinished = _jobFinished
+
+ // If the job is finished, this will be its result. In the case of 0 task jobs (e.g. zero
+ // partition RDDs), we set the jobResult directly to JobSucceeded.
+ private var jobResult: JobResult = if (jobFinished) JobSucceeded else null
+
+ /**
+ * Sends a signal to the DAGScheduler to cancel the job. The cancellation itself is handled
+ * asynchronously. After the low level scheduler cancels all the tasks belonging to this job, it
+ * will fail this job with a SparkException.
+ */
+ def cancel() {
+ dagScheduler.cancelJob(jobId)
}
- override def jobFailed(exception: Exception) {
- synchronized {
- if (jobFinished) {
- throw new UnsupportedOperationException("jobFailed() called on a finished JobWaiter")
- }
- jobFinished = true
- jobResult = JobFailed(exception, None)
+ override def taskSucceeded(index: Int, result: Any): Unit = synchronized {
+ if (_jobFinished) {
+ throw new UnsupportedOperationException("taskSucceeded() called on a finished JobWaiter")
+ }
+ resultHandler(index, result.asInstanceOf[T])
+ finishedTasks += 1
+ if (finishedTasks == totalTasks) {
+ _jobFinished = true
+ jobResult = JobSucceeded
this.notifyAll()
}
}
+ override def jobFailed(exception: Exception): Unit = synchronized {
+ _jobFinished = true
+ jobResult = JobFailed(exception, None)
+ this.notifyAll()
+ }
+
def awaitResult(): JobResult = synchronized {
- while (!jobFinished) {
+ while (!_jobFinished) {
this.wait()
}
return jobResult
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
index 35b32600da..596f9adde9 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/Pool.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
@@ -15,13 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import org.apache.spark.Logging
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
/**
* An Schedulable entity that represent collection of Pools or TaskSetManagers
@@ -43,9 +43,12 @@ private[spark] class Pool(
var runningTasks = 0
var priority = 0
- var stageId = 0
+
+ // A pool's stage id is used to break the tie in scheduling.
+ var stageId = -1
+
var name = poolName
- var parent:Schedulable = null
+ var parent: Pool = null
var taskSetSchedulingAlgorithm: SchedulingAlgorithm = {
schedulingMode match {
@@ -101,14 +104,14 @@ private[spark] class Pool(
return sortedTaskSetQueue
}
- override def increaseRunningTasks(taskNum: Int) {
+ def increaseRunningTasks(taskNum: Int) {
runningTasks += taskNum
if (parent != null) {
parent.increaseRunningTasks(taskNum)
}
}
- override def decreaseRunningTasks(taskNum: Int) {
+ def decreaseRunningTasks(taskNum: Int) {
runningTasks -= taskNum
if (parent != null) {
parent.decreaseRunningTasks(taskNum)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index 2b007cbe82..310ec62ca8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -23,7 +23,7 @@ import java.util.zip.{GZIPInputStream, GZIPOutputStream}
import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.RDDCheckpointData
-import org.apache.spark.util.{MetadataCleaner, TimeStampedHashMap}
+import org.apache.spark.util.{MetadataCleanerType, MetadataCleaner, TimeStampedHashMap}
private[spark] object ResultTask {
@@ -32,23 +32,23 @@ private[spark] object ResultTask {
// expensive on the master node if it needs to launch thousands of tasks.
val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
- val metadataCleaner = new MetadataCleaner("ResultTask", serializedInfoCache.clearOldValues)
+ val metadataCleaner = new MetadataCleaner(MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues)
def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
synchronized {
val old = serializedInfoCache.get(stageId).orNull
if (old != null) {
- return old
+ old
} else {
val out = new ByteArrayOutputStream
- val ser = SparkEnv.get.closureSerializer.newInstance
+ val ser = SparkEnv.get.closureSerializer.newInstance()
val objOut = ser.serializeStream(new GZIPOutputStream(out))
objOut.writeObject(rdd)
objOut.writeObject(func)
objOut.close()
val bytes = out.toByteArray
serializedInfoCache.put(stageId, bytes)
- return bytes
+ bytes
}
}
}
@@ -56,11 +56,11 @@ private[spark] object ResultTask {
def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = {
val loader = Thread.currentThread.getContextClassLoader
val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
- val ser = SparkEnv.get.closureSerializer.newInstance
+ val ser = SparkEnv.get.closureSerializer.newInstance()
val objIn = ser.deserializeStream(in)
val rdd = objIn.readObject().asInstanceOf[RDD[_]]
val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _]
- return (rdd, func)
+ (rdd, func)
}
def clearCache() {
@@ -71,29 +71,37 @@ private[spark] object ResultTask {
}
+/**
+ * A task that sends back the output to the driver application.
+ *
+ * See [[org.apache.spark.scheduler.Task]] for more information.
+ *
+ * @param stageId id of the stage this task belongs to
+ * @param rdd input to func
+ * @param func a function to apply on a partition of the RDD
+ * @param _partitionId index of the number in the RDD
+ * @param locs preferred task execution locations for locality scheduling
+ * @param outputId index of the task in this job (a job can launch tasks on only a subset of the
+ * input RDD's partitions).
+ */
private[spark] class ResultTask[T, U](
stageId: Int,
var rdd: RDD[T],
var func: (TaskContext, Iterator[T]) => U,
- var partition: Int,
+ _partitionId: Int,
@transient locs: Seq[TaskLocation],
- val outputId: Int)
- extends Task[U](stageId) with Externalizable {
+ var outputId: Int)
+ extends Task[U](stageId, _partitionId) with Externalizable {
def this() = this(0, null, null, 0, null, 0)
- var split = if (rdd == null) {
- null
- } else {
- rdd.partitions(partition)
- }
+ var split = if (rdd == null) null else rdd.partitions(partitionId)
@transient private val preferredLocs: Seq[TaskLocation] = {
if (locs == null) Nil else locs.toSet.toSeq
}
- override def run(attemptId: Long): U = {
- val context = new TaskContext(stageId, partition, attemptId)
+ override def runTask(context: TaskContext): U = {
metrics = Some(context.taskMetrics)
try {
func(context, rdd.iterator(split, context))
@@ -104,17 +112,17 @@ private[spark] class ResultTask[T, U](
override def preferredLocations: Seq[TaskLocation] = preferredLocs
- override def toString = "ResultTask(" + stageId + ", " + partition + ")"
+ override def toString = "ResultTask(" + stageId + ", " + partitionId + ")"
override def writeExternal(out: ObjectOutput) {
RDDCheckpointData.synchronized {
- split = rdd.partitions(partition)
+ split = rdd.partitions(partitionId)
out.writeInt(stageId)
val bytes = ResultTask.serializeInfo(
stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _])
out.writeInt(bytes.length)
out.write(bytes)
- out.writeInt(partition)
+ out.writeInt(partitionId)
out.writeInt(outputId)
out.writeLong(epoch)
out.writeObject(split)
@@ -129,8 +137,8 @@ private[spark] class ResultTask[T, U](
val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes)
rdd = rdd_.asInstanceOf[RDD[T]]
func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U]
- partition = in.readInt()
- val outputId = in.readInt()
+ partitionId = in.readInt()
+ outputId = in.readInt()
epoch = in.readLong()
split = in.readObject().asInstanceOf[Partition]
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
index f4726450ec..1c7ea2dccc 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/Schedulable.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import scala.collection.mutable.ArrayBuffer
/**
@@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer
* there are two type of Schedulable entities(Pools and TaskSetManagers)
*/
private[spark] trait Schedulable {
- var parent: Schedulable
+ var parent: Pool
// child queues
def schedulableQueue: ArrayBuffer[Schedulable]
def schedulingMode: SchedulingMode
@@ -36,8 +36,6 @@ private[spark] trait Schedulable {
def stageId: Int
def name: String
- def increaseRunningTasks(taskNum: Int): Unit
- def decreaseRunningTasks(taskNum: Int): Unit
def addSchedulable(schedulable: Schedulable): Unit
def removeSchedulable(schedulable: Schedulable): Unit
def getSchedulableByName(name: String): Schedulable
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
index d04eeb6b98..356fe56bf3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
@@ -15,16 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
-import java.io.{File, FileInputStream, FileOutputStream, FileNotFoundException}
-import java.util.Properties
-
-import scala.xml.XML
+import java.io.{FileInputStream, InputStream}
+import java.util.{NoSuchElementException, Properties}
import org.apache.spark.Logging
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import scala.xml.XML
/**
* An interface to build Schedulable tree
@@ -32,7 +30,10 @@ import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
* addTaskSetManager: build the leaf nodes(TaskSetManagers)
*/
private[spark] trait SchedulableBuilder {
+ def rootPool: Pool
+
def buildPools()
+
def addTaskSetManager(manager: Schedulable, properties: Properties)
}
@@ -51,8 +52,9 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool)
private[spark] class FairSchedulableBuilder(val rootPool: Pool)
extends SchedulableBuilder with Logging {
- val schedulerAllocFile = System.getProperty("spark.fairscheduler.allocation.file")
- val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.cluster.fair.pool"
+ val schedulerAllocFile = Option(System.getProperty("spark.scheduler.allocation.file"))
+ val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml"
+ val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool"
val DEFAULT_POOL_NAME = "default"
val MINIMUM_SHARES_PROPERTY = "minShare"
val SCHEDULING_MODE_PROPERTY = "schedulingMode"
@@ -60,52 +62,30 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool)
val POOL_NAME_PROPERTY = "@name"
val POOLS_PROPERTY = "pool"
val DEFAULT_SCHEDULING_MODE = SchedulingMode.FIFO
- val DEFAULT_MINIMUM_SHARE = 2
+ val DEFAULT_MINIMUM_SHARE = 0
val DEFAULT_WEIGHT = 1
override def buildPools() {
- if (schedulerAllocFile != null) {
- val file = new File(schedulerAllocFile)
- if (file.exists()) {
- val xml = XML.loadFile(file)
- for (poolNode <- (xml \\ POOLS_PROPERTY)) {
-
- val poolName = (poolNode \ POOL_NAME_PROPERTY).text
- var schedulingMode = DEFAULT_SCHEDULING_MODE
- var minShare = DEFAULT_MINIMUM_SHARE
- var weight = DEFAULT_WEIGHT
-
- val xmlSchedulingMode = (poolNode \ SCHEDULING_MODE_PROPERTY).text
- if (xmlSchedulingMode != "") {
- try {
- schedulingMode = SchedulingMode.withName(xmlSchedulingMode)
- } catch {
- case e: Exception => logInfo("Error xml schedulingMode, using default schedulingMode")
- }
- }
-
- val xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text
- if (xmlMinShare != "") {
- minShare = xmlMinShare.toInt
- }
-
- val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text
- if (xmlWeight != "") {
- weight = xmlWeight.toInt
- }
-
- val pool = new Pool(poolName, schedulingMode, minShare, weight)
- rootPool.addSchedulable(pool)
- logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format(
- poolName, schedulingMode, minShare, weight))
+ var is: Option[InputStream] = None
+ try {
+ is = Option {
+ schedulerAllocFile.map { f =>
+ new FileInputStream(f)
+ }.getOrElse {
+ getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE)
}
- } else {
- throw new java.io.FileNotFoundException(
- "Fair scheduler allocation file not found: " + schedulerAllocFile)
}
+
+ is.foreach { i => buildFairSchedulerPool(i) }
+ } finally {
+ is.foreach(_.close())
}
// finally create "default" pool
+ buildDefaultPool()
+ }
+
+ private def buildDefaultPool() {
if (rootPool.getSchedulableByName(DEFAULT_POOL_NAME) == null) {
val pool = new Pool(DEFAULT_POOL_NAME, DEFAULT_SCHEDULING_MODE,
DEFAULT_MINIMUM_SHARE, DEFAULT_WEIGHT)
@@ -115,6 +95,42 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool)
}
}
+ private def buildFairSchedulerPool(is: InputStream) {
+ val xml = XML.load(is)
+ for (poolNode <- (xml \\ POOLS_PROPERTY)) {
+
+ val poolName = (poolNode \ POOL_NAME_PROPERTY).text
+ var schedulingMode = DEFAULT_SCHEDULING_MODE
+ var minShare = DEFAULT_MINIMUM_SHARE
+ var weight = DEFAULT_WEIGHT
+
+ val xmlSchedulingMode = (poolNode \ SCHEDULING_MODE_PROPERTY).text
+ if (xmlSchedulingMode != "") {
+ try {
+ schedulingMode = SchedulingMode.withName(xmlSchedulingMode)
+ } catch {
+ case e: NoSuchElementException =>
+ logWarning("Error xml schedulingMode, using default schedulingMode")
+ }
+ }
+
+ val xmlMinShare = (poolNode \ MINIMUM_SHARES_PROPERTY).text
+ if (xmlMinShare != "") {
+ minShare = xmlMinShare.toInt
+ }
+
+ val xmlWeight = (poolNode \ WEIGHT_PROPERTY).text
+ if (xmlWeight != "") {
+ weight = xmlWeight.toInt
+ }
+
+ val pool = new Pool(poolName, schedulingMode, minShare, weight)
+ rootPool.addSchedulable(pool)
+ logInfo("Created pool %s, schedulingMode: %s, minShare: %d, weight: %d".format(
+ poolName, schedulingMode, minShare, weight))
+ }
+ }
+
override def addTaskSetManager(manager: Schedulable, properties: Properties) {
var poolName = DEFAULT_POOL_NAME
var parentPool = rootPool.getSchedulableByName(poolName)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala
index cbeed4731a..3418640b8c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingAlgorithm.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
/**
* An interface for sort algorithm
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
index 34811389a0..0a786deb16 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulingMode.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
/**
* "FAIR" and "FIFO" determines which policy is used
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 764775fede..802791797a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -25,7 +25,7 @@ import scala.collection.mutable.HashMap
import org.apache.spark._
import org.apache.spark.executor.ShuffleWriteMetrics
import org.apache.spark.storage._
-import org.apache.spark.util.{TimeStampedHashMap, MetadataCleaner}
+import org.apache.spark.util.{MetadataCleanerType, TimeStampedHashMap, MetadataCleaner}
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.RDDCheckpointData
@@ -37,7 +37,7 @@ private[spark] object ShuffleMapTask {
// expensive on the master node if it needs to launch thousands of tasks.
val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]]
- val metadataCleaner = new MetadataCleaner("ShuffleMapTask", serializedInfoCache.clearOldValues)
+ val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues)
def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
synchronized {
@@ -53,7 +53,7 @@ private[spark] object ShuffleMapTask {
objOut.close()
val bytes = out.toByteArray
serializedInfoCache.put(stageId, bytes)
- return bytes
+ bytes
}
}
}
@@ -66,7 +66,7 @@ private[spark] object ShuffleMapTask {
val objIn = ser.deserializeStream(in)
val rdd = objIn.readObject().asInstanceOf[RDD[_]]
val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]]
- return (rdd, dep)
+ (rdd, dep)
}
}
@@ -75,7 +75,7 @@ private[spark] object ShuffleMapTask {
val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
val objIn = new ObjectInputStream(in)
val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap
- return (HashMap(set.toSeq: _*))
+ HashMap(set.toSeq: _*)
}
def clearCache() {
@@ -85,13 +85,25 @@ private[spark] object ShuffleMapTask {
}
}
+/**
+ * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner
+ * specified in the ShuffleDependency).
+ *
+ * See [[org.apache.spark.scheduler.Task]] for more information.
+ *
+ * @param stageId id of the stage this task belongs to
+ * @param rdd the final RDD in this stage
+ * @param dep the ShuffleDependency
+ * @param _partitionId index of the number in the RDD
+ * @param locs preferred task execution locations for locality scheduling
+ */
private[spark] class ShuffleMapTask(
stageId: Int,
var rdd: RDD[_],
var dep: ShuffleDependency[_,_],
- var partition: Int,
+ _partitionId: Int,
@transient private var locs: Seq[TaskLocation])
- extends Task[MapStatus](stageId)
+ extends Task[MapStatus](stageId, _partitionId)
with Externalizable
with Logging {
@@ -101,16 +113,16 @@ private[spark] class ShuffleMapTask(
if (locs == null) Nil else locs.toSet.toSeq
}
- var split = if (rdd == null) null else rdd.partitions(partition)
+ var split = if (rdd == null) null else rdd.partitions(partitionId)
override def writeExternal(out: ObjectOutput) {
RDDCheckpointData.synchronized {
- split = rdd.partitions(partition)
+ split = rdd.partitions(partitionId)
out.writeInt(stageId)
val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
out.writeInt(bytes.length)
out.write(bytes)
- out.writeInt(partition)
+ out.writeInt(partitionId)
out.writeLong(epoch)
out.writeObject(split)
}
@@ -124,16 +136,14 @@ private[spark] class ShuffleMapTask(
val (rdd_, dep_) = ShuffleMapTask.deserializeInfo(stageId, bytes)
rdd = rdd_
dep = dep_
- partition = in.readInt()
+ partitionId = in.readInt()
epoch = in.readLong()
split = in.readObject().asInstanceOf[Partition]
}
- override def run(attemptId: Long): MapStatus = {
+ override def runTask(context: TaskContext): MapStatus = {
val numOutputSplits = dep.partitioner.numPartitions
-
- val taskContext = new TaskContext(stageId, partition, attemptId)
- metrics = Some(taskContext.taskMetrics)
+ metrics = Some(context.taskMetrics)
val blockManager = SparkEnv.get.blockManager
var shuffle: ShuffleBlocks = null
@@ -143,10 +153,10 @@ private[spark] class ShuffleMapTask(
// Obtain all the block writers for shuffle blocks.
val ser = SparkEnv.get.serializerManager.get(dep.serializerClass)
shuffle = blockManager.shuffleBlockManager.forShuffle(dep.shuffleId, numOutputSplits, ser)
- buckets = shuffle.acquireWriters(partition)
+ buckets = shuffle.acquireWriters(partitionId)
// Write the map output to its associated buckets.
- for (elem <- rdd.iterator(split, taskContext)) {
+ for (elem <- rdd.iterator(split, context)) {
val pair = elem.asInstanceOf[Product2[Any, Any]]
val bucketId = dep.partitioner.getPartition(pair._1)
buckets.writers(bucketId).write(pair)
@@ -167,7 +177,7 @@ private[spark] class ShuffleMapTask(
shuffleMetrics.shuffleBytesWritten = totalBytes
metrics.get.shuffleWriteMetrics = Some(shuffleMetrics)
- return new MapStatus(blockManager.blockManagerId, compressedSizes)
+ new MapStatus(blockManager.blockManagerId, compressedSizes)
} catch { case e: Exception =>
// If there is an exception from running the task, revert the partial writes
// and throw the exception upstream to Spark.
@@ -181,11 +191,11 @@ private[spark] class ShuffleMapTask(
shuffle.releaseWriters(buckets)
}
// Execute the callbacks on task completion.
- taskContext.executeOnCompleteCallbacks()
+ context.executeOnCompleteCallbacks()
}
}
override def preferredLocations: Seq[TaskLocation] = preferredLocs
- override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partition)
+ override def toString = "ShuffleMapTask(%d, %d)".format(stageId, partitionId)
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index c3cf4b8907..466baf9913 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -18,7 +18,6 @@
package org.apache.spark.scheduler
import java.util.Properties
-import org.apache.spark.scheduler.cluster.TaskInfo
import org.apache.spark.util.{Utils, Distribution}
import org.apache.spark.{Logging, SparkContext, TaskEndReason}
import org.apache.spark.executor.TaskMetrics
@@ -55,7 +54,7 @@ trait SparkListener {
/**
* Called when a task starts
*/
- def onTaskStart(taskEnd: SparkListenerTaskStart) { }
+ def onTaskStart(taskStart: SparkListenerTaskStart) { }
/**
* Called when a task ends
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index a65e1ecd6d..4d3e4a17ba 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -70,5 +70,23 @@ private[spark] class SparkListenerBus() extends Logging {
queueFullErrorMessageLogged = true
}
}
+
+ /**
+ * Waits until there are no more events in the queue, or until the specified time has elapsed.
+ * Used for testing only. Returns true if the queue has emptied and false is the specified time
+ * elapsed before the queue emptied.
+ */
+ def waitUntilEmpty(timeoutMillis: Int): Boolean = {
+ val finishTime = System.currentTimeMillis + timeoutMillis
+ while (!eventQueue.isEmpty()) {
+ if (System.currentTimeMillis > finishTime) {
+ return false
+ }
+ /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify
+ * add overhead in the general case. */
+ Thread.sleep(10)
+ }
+ return true
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index 72cb1c9ce8..b6f11969e5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -17,8 +17,8 @@
package org.apache.spark.scheduler
-import org.apache.spark.scheduler.cluster.TaskInfo
import scala.collection._
+
import org.apache.spark.executor.TaskMetrics
case class StageInfo(
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index 598d91752a..1fe0d0e4e2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -17,25 +17,74 @@
package org.apache.spark.scheduler
-import org.apache.spark.serializer.SerializerInstance
import java.io.{DataInputStream, DataOutputStream}
import java.nio.ByteBuffer
-import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
-import org.apache.spark.util.ByteBufferInputStream
+
import scala.collection.mutable.HashMap
+
+import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
+
+import org.apache.spark.TaskContext
import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.serializer.SerializerInstance
+import org.apache.spark.util.ByteBufferInputStream
+
/**
- * A task to execute on a worker node.
+ * A unit of execution. We have two kinds of Task's in Spark:
+ * - [[org.apache.spark.scheduler.ShuffleMapTask]]
+ * - [[org.apache.spark.scheduler.ResultTask]]
+ *
+ * A Spark job consists of one or more stages. The very last stage in a job consists of multiple
+ * ResultTask's, while earlier stages consist of ShuffleMapTasks. A ResultTask executes the task
+ * and sends the task output back to the driver application. A ShuffleMapTask executes the task
+ * and divides the task output to multiple buckets (based on the task's partitioner).
+ *
+ * @param stageId id of the stage this task belongs to
+ * @param partitionId index of the number in the RDD
*/
-private[spark] abstract class Task[T](val stageId: Int) extends Serializable {
- def run(attemptId: Long): T
+private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable {
+
+ def run(attemptId: Long): T = {
+ context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false)
+ if (_killed) {
+ kill()
+ }
+ runTask(context)
+ }
+
+ def runTask(context: TaskContext): T
+
def preferredLocations: Seq[TaskLocation] = Nil
- var epoch: Long = -1 // Map output tracker epoch. Will be set by TaskScheduler.
+ // Map output tracker epoch. Will be set by TaskScheduler.
+ var epoch: Long = -1
var metrics: Option[TaskMetrics] = None
+ // Task context, to be initialized in run().
+ @transient protected var context: TaskContext = _
+
+ // A flag to indicate whether the task is killed. This is used in case context is not yet
+ // initialized when kill() is invoked.
+ @volatile @transient private var _killed = false
+
+ /**
+ * Whether the task has been killed.
+ */
+ def killed: Boolean = _killed
+
+ /**
+ * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark
+ * code and user code to properly handle the flag. This function should be idempotent so it can
+ * be called multiple times.
+ */
+ def kill() {
+ _killed = true
+ if (context != null) {
+ context.interrupted = true
+ }
+ }
}
/**
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
index 309ac2f6c9..5190d234d4 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskDescription.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import java.nio.ByteBuffer
import org.apache.spark.util.SerializableBuffer
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
index 9685fb1a67..7c2a422aff 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import org.apache.spark.util.Utils
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
index 5d4130e14a..47b0f387aa 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskLocality.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
private[spark] object TaskLocality
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 5c7e5bb977..7e468d0d67 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
@@ -24,14 +24,20 @@ import org.apache.spark.executor.TaskMetrics
import org.apache.spark.{SparkEnv}
import java.nio.ByteBuffer
import org.apache.spark.util.Utils
+import org.apache.spark.storage.BlockId
// Task result. Also contains updates to accumulator variables.
-// TODO: Use of distributed cache to return result is a hack to get around
-// what seems to be a bug with messages over 60KB in libprocess; fix it
+private[spark] sealed trait TaskResult[T]
+
+/** A reference to a DirectTaskResult that has been stored in the worker's BlockManager. */
+private[spark]
+case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Serializable
+
+/** A TaskResult that contains the task's return value and accumulator updates. */
private[spark]
-class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics)
- extends Externalizable
-{
+class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics)
+ extends TaskResult[T] with Externalizable {
+
def this() = this(null.asInstanceOf[T], null, null)
override def writeExternal(out: ObjectOutput) {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
index 63be8ba3f5..6a51efe8d6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -17,10 +17,11 @@
package org.apache.spark.scheduler
-import org.apache.spark.scheduler.cluster.Pool
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
+
/**
* Low-level task scheduler interface, implemented by both ClusterScheduler and LocalScheduler.
+ * Each TaskScheduler schedulers task for a single SparkContext.
* These schedulers get sets of tasks submitted to them from the DAGScheduler for each stage,
* and are responsible for sending the tasks to the cluster, running them, retrying if there
* are failures, and mitigating stragglers. They return events to the DAGScheduler through
@@ -44,6 +45,9 @@ private[spark] trait TaskScheduler {
// Submit a sequence of tasks to run.
def submitTasks(taskSet: TaskSet): Unit
+ // Cancel a stage.
+ def cancelTasks(stageId: Int)
+
// Set a listener for upcalls. This is guaranteed to be set before submitTasks is called.
def setListener(listener: TaskSchedulerListener): Unit
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
index 83be051c1a..593fa9fb93 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerListener.scala
@@ -17,7 +17,6 @@
package org.apache.spark.scheduler
-import org.apache.spark.scheduler.cluster.TaskInfo
import scala.collection.mutable.Map
import org.apache.spark.TaskEndReason
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
index c3ad325156..03bf760837 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala
@@ -31,5 +31,9 @@ private[spark] class TaskSet(
val properties: Properties) {
val id: String = stageId + "." + attempt
+ def kill() {
+ tasks.foreach(_.kill())
+ }
+
override def toString: String = "TaskSet " + id
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 648a3ef922..90f6bcefac 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import java.nio.ByteBuffer
import org.apache.spark.TaskState.TaskState
-import org.apache.spark.scheduler.TaskSet
/**
* Tracks and schedules the tasks within a single TaskSet. This class keeps track of the status of
@@ -45,7 +44,5 @@ private[spark] trait TaskSetManager extends Schedulable {
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription]
- def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer)
-
def error(message: String)
}
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 3196ab5022..7a72ff0474 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
@@ -17,7 +17,9 @@
package org.apache.spark.scheduler.cluster
-import java.lang.{Boolean => JBoolean}
+import java.nio.ByteBuffer
+import java.util.concurrent.atomic.AtomicLong
+import java.util.{TimerTask, Timer}
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
@@ -26,10 +28,7 @@ import scala.collection.mutable.HashSet
import org.apache.spark._
import org.apache.spark.TaskState.TaskState
import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
-import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicLong
-import java.util.{TimerTask, Timer}
+import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
/**
* The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call
@@ -55,7 +54,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
// Threshold above which we warn user initial TaskSet may be starved
val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong
- val activeTaskSets = new HashMap[String, TaskSetManager]
+ // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized
+ // on this class.
+ val activeTaskSets = new HashMap[String, ClusterTaskSetManager]
val taskIdToTaskSetId = new HashMap[Long, String]
val taskIdToExecutorId = new HashMap[Long, String]
@@ -65,7 +66,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
@volatile private var hasLaunchedTask = false
private val starvationTimer = new Timer(true)
- // Incrementing Mesos task IDs
+ // Incrementing task IDs
val nextTaskId = new AtomicLong(0)
// Which executor IDs we have executors on
@@ -77,12 +78,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
private val executorIdToHost = new HashMap[String, String]
- // JAR server, if any JARs were added by the user to the SparkContext
- var jarServer: HttpServer = null
-
- // URIs of JARs to pass to executor
- var jarUris: String = ""
-
// Listener object to pass upcalls into
var listener: TaskSchedulerListener = null
@@ -94,7 +89,10 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
var rootPool: Pool = null
// default scheduler is FIFO
val schedulingMode: SchedulingMode = SchedulingMode.withName(
- System.getProperty("spark.cluster.schedulingmode", "FIFO"))
+ System.getProperty("spark.scheduler.mode", "FIFO"))
+
+ // This is a var so that we can reset it for testing purposes.
+ private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this)
override def setListener(listener: TaskSchedulerListener) {
this.listener = listener
@@ -166,8 +164,31 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
backend.reviveOffers()
}
- def taskSetFinished(manager: TaskSetManager) {
- this.synchronized {
+ override def cancelTasks(stageId: Int): Unit = synchronized {
+ logInfo("Cancelling stage " + stageId)
+ activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) =>
+ // There are two possible cases here:
+ // 1. The task set manager has been created and some tasks have been scheduled.
+ // In this case, send a kill signal to the executors to kill the task and then abort
+ // the stage.
+ // 2. The task set manager has been created but no tasks has been scheduled. In this case,
+ // simply abort the stage.
+ val taskIds = taskSetTaskIds(tsm.taskSet.id)
+ if (taskIds.size > 0) {
+ taskIds.foreach { tid =>
+ val execId = taskIdToExecutorId(tid)
+ backend.killTask(tid, execId)
+ }
+ }
+ tsm.error("Stage %d was cancelled".format(stageId))
+ }
+ }
+
+ def taskSetFinished(manager: TaskSetManager): Unit = synchronized {
+ // Check to see if the given task set has been removed. This is possible in the case of
+ // multiple unrecoverable task failures (e.g. if the entire task set is killed when it has
+ // more than one running tasks).
+ if (activeTaskSets.contains(manager.taskSet.id)) {
activeTaskSets -= manager.taskSet.id
manager.parent.removeSchedulable(manager)
logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name))
@@ -234,7 +255,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
- var taskSetToUpdate: Option[TaskSetManager] = None
var failedExecutor: Option[String] = None
var taskFailed = false
synchronized {
@@ -249,9 +269,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
taskIdToTaskSetId.get(tid) match {
case Some(taskSetId) =>
- if (activeTaskSets.contains(taskSetId)) {
- taskSetToUpdate = Some(activeTaskSets(taskSetId))
- }
if (TaskState.isFinished(state)) {
taskIdToTaskSetId.remove(tid)
if (taskSetTaskIds.contains(taskSetId)) {
@@ -262,6 +279,15 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
if (state == TaskState.FAILED) {
taskFailed = true
}
+ activeTaskSets.get(taskSetId).foreach { taskSet =>
+ if (state == TaskState.FINISHED) {
+ taskSet.removeRunningTask(tid)
+ taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData)
+ } else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) {
+ taskSet.removeRunningTask(tid)
+ taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData)
+ }
+ }
case None =>
logInfo("Ignoring update from TID " + tid + " because its task set is gone")
}
@@ -269,10 +295,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
case e: Exception => logError("Exception in statusUpdate", e)
}
}
- // Update the task set and DAGScheduler without holding a lock on this, since that can deadlock
- if (taskSetToUpdate != None) {
- taskSetToUpdate.get.statusUpdate(tid, state, serializedData)
- }
+ // Update the DAGScheduler without holding a lock on this, since that can deadlock
if (failedExecutor != None) {
listener.executorLost(failedExecutor.get)
backend.reviveOffers()
@@ -283,6 +306,25 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
}
+ def handleSuccessfulTask(
+ taskSetManager: ClusterTaskSetManager,
+ tid: Long,
+ taskResult: DirectTaskResult[_]) = synchronized {
+ taskSetManager.handleSuccessfulTask(tid, taskResult)
+ }
+
+ def handleFailedTask(
+ taskSetManager: ClusterTaskSetManager,
+ tid: Long,
+ taskState: TaskState,
+ reason: Option[TaskEndReason]) = synchronized {
+ taskSetManager.handleFailedTask(tid, taskState, reason)
+ if (taskState == TaskState.FINISHED) {
+ // The task finished successfully but the result was lost, so we should revive offers.
+ backend.reviveOffers()
+ }
+ }
+
def error(message: String) {
synchronized {
if (activeTaskSets.size > 0) {
@@ -308,8 +350,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
if (backend != null) {
backend.stop()
}
- if (jarServer != null) {
- jarServer.stop()
+ if (taskResultGetter != null) {
+ taskResultGetter.stop()
}
// sleeping for an arbitrary 5 seconds : to ensure that messages are sent out.
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 1b31c8c57e..e1366e0627 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
@@ -17,8 +17,7 @@
package org.apache.spark.scheduler.cluster
-import java.nio.ByteBuffer
-import java.util.{Arrays, NoSuchElementException}
+import java.util.Arrays
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
@@ -26,14 +25,10 @@ import scala.collection.mutable.HashSet
import scala.math.max
import scala.math.min
-import org.apache.spark.{FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskState}
-import org.apache.spark.{ExceptionFailure, SparkException, TaskResultTooBigFailure}
+import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv,
+ Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState}
import org.apache.spark.TaskState.TaskState
import org.apache.spark.scheduler._
-import scala.Some
-import org.apache.spark.FetchFailed
-import org.apache.spark.ExceptionFailure
-import org.apache.spark.TaskResultTooBigFailure
import org.apache.spark.util.{SystemClock, Clock}
@@ -71,18 +66,20 @@ private[spark] class ClusterTaskSetManager(
val tasks = taskSet.tasks
val numTasks = tasks.length
val copiesRunning = new Array[Int](numTasks)
- val finished = new Array[Boolean](numTasks)
+ val successful = new Array[Boolean](numTasks)
val numFailures = new Array[Int](numTasks)
val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil)
- var tasksFinished = 0
+ var tasksSuccessful = 0
var weight = 1
var minShare = 0
- var runningTasks = 0
var priority = taskSet.priority
var stageId = taskSet.stageId
var name = "TaskSet_"+taskSet.stageId.toString
- var parent: Schedulable = null
+ var parent: Pool = null
+
+ var runningTasks = 0
+ private val runningTasksSet = new HashSet[Long]
// Set of pending tasks for each executor. These collections are actually
// treated as stacks, in which new tasks are added to the end of the
@@ -223,7 +220,7 @@ private[spark] class ClusterTaskSetManager(
while (!list.isEmpty) {
val index = list.last
list.trimEnd(1)
- if (copiesRunning(index) == 0 && !finished(index)) {
+ if (copiesRunning(index) == 0 && !successful(index)) {
return Some(index)
}
}
@@ -243,7 +240,7 @@ private[spark] class ClusterTaskSetManager(
private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value)
: Option[(Int, TaskLocality.Value)] =
{
- speculatableTasks.retain(index => !finished(index)) // Remove finished tasks from set
+ speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set
if (!speculatableTasks.isEmpty) {
// Check for process-local or preference-less tasks; note that tasks can be process-local
@@ -335,7 +332,7 @@ private[spark] class ClusterTaskSetManager(
}
/**
- * Respond to an offer of a single slave from the scheduler by finding a task
+ * Respond to an offer of a single executor from the scheduler by finding a task
*/
override def resourceOffer(
execId: String,
@@ -344,7 +341,7 @@ private[spark] class ClusterTaskSetManager(
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription] =
{
- if (tasksFinished < numTasks && availableCpus >= CPUS_PER_TASK) {
+ if (tasksSuccessful < numTasks && availableCpus >= CPUS_PER_TASK) {
val curTime = clock.getTime()
var allowedLocality = getAllowedLocalityLevel(curTime)
@@ -358,7 +355,7 @@ private[spark] class ClusterTaskSetManager(
val task = tasks(index)
val taskId = sched.newTaskId()
// Figure out whether this should count as a preferred launch
- logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format(
+ logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format(
taskSet.id, index, taskId, execId, host, taskLocality))
// Do various bookkeeping
copiesRunning(index) += 1
@@ -375,7 +372,7 @@ private[spark] class ClusterTaskSetManager(
val serializedTask = Task.serializeWithDependencies(
task, sched.sc.addedFiles, sched.sc.addedJars, ser)
val timeTaken = clock.getTime() - startTime
- increaseRunningTasks(1)
+ addRunningTask(taskId)
logInfo("Serialized task %s:%d as %d bytes in %d ms".format(
taskSet.id, index, serializedTask.limit, timeTaken))
val taskName = "task %s:%d".format(taskSet.id, index)
@@ -417,130 +414,103 @@ private[spark] class ClusterTaskSetManager(
index
}
- /** Called by cluster scheduler when one of our tasks changes state */
- override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
- SparkEnv.set(env)
- state match {
- case TaskState.FINISHED =>
- taskFinished(tid, state, serializedData)
- case TaskState.LOST =>
- taskLost(tid, state, serializedData)
- case TaskState.FAILED =>
- taskLost(tid, state, serializedData)
- case TaskState.KILLED =>
- taskLost(tid, state, serializedData)
- case _ =>
- }
- }
-
- def taskStarted(task: Task[_], info: TaskInfo) {
+ private def taskStarted(task: Task[_], info: TaskInfo) {
sched.listener.taskStarted(task, info)
}
- def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) {
+ /**
+ * Marks the task as successful and notifies the listener that a task has ended.
+ */
+ def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = {
val info = taskInfos(tid)
- if (info.failed) {
- // We might get two task-lost messages for the same task in coarse-grained Mesos mode,
- // or even from Mesos itself when acks get delayed.
- return
- }
val index = info.index
info.markSuccessful()
- decreaseRunningTasks(1)
- if (!finished(index)) {
- tasksFinished += 1
+ removeRunningTask(tid)
+ if (!successful(index)) {
logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format(
- tid, info.duration, info.host, tasksFinished, numTasks))
- // Deserialize task result and pass it to the scheduler
- try {
- val result = ser.deserialize[TaskResult[_]](serializedData)
- result.metrics.resultSize = serializedData.limit()
- sched.listener.taskEnded(
- tasks(index), Success, result.value, result.accumUpdates, info, result.metrics)
- } catch {
- case cnf: ClassNotFoundException =>
- val loader = Thread.currentThread().getContextClassLoader
- throw new SparkException("ClassNotFound with classloader: " + loader, cnf)
- case ex => throw ex
- }
- // Mark finished and stop if we've finished all the tasks
- finished(index) = true
- if (tasksFinished == numTasks) {
+ tid, info.duration, info.host, tasksSuccessful, numTasks))
+ sched.listener.taskEnded(
+ tasks(index), Success, result.value, result.accumUpdates, info, result.metrics)
+
+ // Mark successful and stop if all the tasks have succeeded.
+ tasksSuccessful += 1
+ successful(index) = true
+ if (tasksSuccessful == numTasks) {
sched.taskSetFinished(this)
}
} else {
- logInfo("Ignoring task-finished event for TID " + tid +
- " because task " + index + " is already finished")
+ logInfo("Ignorning task-finished event for TID " + tid + " because task " +
+ index + " has already completed successfully")
}
}
- def taskLost(tid: Long, state: TaskState, serializedData: ByteBuffer) {
+ /**
+ * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the listener.
+ */
+ def handleFailedTask(tid: Long, state: TaskState, reason: Option[TaskEndReason]) {
val info = taskInfos(tid)
if (info.failed) {
- // We might get two task-lost messages for the same task in coarse-grained Mesos mode,
- // or even from Mesos itself when acks get delayed.
return
}
+ removeRunningTask(tid)
val index = info.index
info.markFailed()
- decreaseRunningTasks(1)
- if (!finished(index)) {
+ if (!successful(index)) {
logInfo("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index))
copiesRunning(index) -= 1
// Check if the problem is a map output fetch failure. In that case, this
// task will never succeed on any node, so tell the scheduler about it.
- if (serializedData != null && serializedData.limit() > 0) {
- val reason = ser.deserialize[TaskEndReason](serializedData, getClass.getClassLoader)
- reason match {
- case fetchFailed: FetchFailed =>
- logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress)
- sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null)
- finished(index) = true
- tasksFinished += 1
- sched.taskSetFinished(this)
- decreaseRunningTasks(runningTasks)
- return
-
- case taskResultTooBig: TaskResultTooBigFailure =>
- logInfo("Loss was due to task %s result exceeding Akka frame size; aborting job".format(
- tid))
- abort("Task %s result exceeded Akka frame size".format(tid))
- return
-
- case ef: ExceptionFailure =>
- sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
- val key = ef.description
- val now = clock.getTime()
- val (printFull, dupCount) = {
- if (recentExceptions.contains(key)) {
- val (dupCount, printTime) = recentExceptions(key)
- if (now - printTime > EXCEPTION_PRINT_INTERVAL) {
- recentExceptions(key) = (0, now)
- (true, 0)
- } else {
- recentExceptions(key) = (dupCount + 1, printTime)
- (false, dupCount + 1)
- }
- } else {
+ reason.foreach {
+ case fetchFailed: FetchFailed =>
+ logInfo("Loss was due to fetch failure from " + fetchFailed.bmAddress)
+ sched.listener.taskEnded(tasks(index), fetchFailed, null, null, info, null)
+ successful(index) = true
+ tasksSuccessful += 1
+ sched.taskSetFinished(this)
+ removeAllRunningTasks()
+ return
+
+ case TaskKilled =>
+ logInfo("Task %d was killed.".format(tid))
+ sched.listener.taskEnded(tasks(index), reason.get, null, null, info, null)
+ return
+
+ case ef: ExceptionFailure =>
+ sched.listener.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
+ val key = ef.description
+ val now = clock.getTime()
+ val (printFull, dupCount) = {
+ if (recentExceptions.contains(key)) {
+ val (dupCount, printTime) = recentExceptions(key)
+ if (now - printTime > EXCEPTION_PRINT_INTERVAL) {
recentExceptions(key) = (0, now)
(true, 0)
+ } else {
+ recentExceptions(key) = (dupCount + 1, printTime)
+ (false, dupCount + 1)
}
- }
- if (printFull) {
- val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString))
- logInfo("Loss was due to %s\n%s\n%s".format(
- ef.className, ef.description, locs.mkString("\n")))
} else {
- logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount))
+ recentExceptions(key) = (0, now)
+ (true, 0)
}
+ }
+ if (printFull) {
+ val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString))
+ logInfo("Loss was due to %s\n%s\n%s".format(
+ ef.className, ef.description, locs.mkString("\n")))
+ } else {
+ logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount))
+ }
- case _ => {}
- }
+ case TaskResultLost =>
+ logInfo("Lost result for TID %s on host %s".format(tid, info.host))
+ sched.listener.taskEnded(tasks(index), TaskResultLost, null, null, info, null)
+
+ case _ => {}
}
// On non-fetch failures, re-enqueue the task as pending for a max number of retries
addPendingTask(index)
- // Count failed attempts only on FAILED and LOST state (not on KILLED)
- if (state == TaskState.FAILED || state == TaskState.LOST) {
+ 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(
@@ -564,22 +534,36 @@ private[spark] class ClusterTaskSetManager(
causeOfFailure = message
// TODO: Kill running tasks if we were not terminated due to a Mesos error
sched.listener.taskSetFailed(taskSet, message)
- decreaseRunningTasks(runningTasks)
+ removeAllRunningTasks()
sched.taskSetFinished(this)
}
- override def increaseRunningTasks(taskNum: Int) {
- runningTasks += taskNum
- if (parent != null) {
- parent.increaseRunningTasks(taskNum)
+ /** If the given task ID is not in the set of running tasks, adds it.
+ *
+ * Used to keep track of the number of running tasks, for enforcing scheduling policies.
+ */
+ def addRunningTask(tid: Long) {
+ if (runningTasksSet.add(tid) && parent != null) {
+ parent.increaseRunningTasks(1)
+ }
+ runningTasks = runningTasksSet.size
+ }
+
+ /** If the given task ID is in the set of running tasks, removes it. */
+ def removeRunningTask(tid: Long) {
+ if (runningTasksSet.remove(tid) && parent != null) {
+ parent.decreaseRunningTasks(1)
}
+ runningTasks = runningTasksSet.size
}
- override def decreaseRunningTasks(taskNum: Int) {
- runningTasks -= taskNum
+ private def removeAllRunningTasks() {
+ val numRunningTasks = runningTasksSet.size
+ runningTasksSet.clear()
if (parent != null) {
- parent.decreaseRunningTasks(taskNum)
+ parent.decreaseRunningTasks(numRunningTasks)
}
+ runningTasks = 0
}
override def getSchedulableByName(name: String): Schedulable = {
@@ -615,10 +599,10 @@ private[spark] class ClusterTaskSetManager(
if (tasks(0).isInstanceOf[ShuffleMapTask]) {
for ((tid, info) <- taskInfos if info.executorId == execId) {
val index = taskInfos(tid).index
- if (finished(index)) {
- finished(index) = false
+ if (successful(index)) {
+ successful(index) = false
copiesRunning(index) -= 1
- tasksFinished -= 1
+ tasksSuccessful -= 1
addPendingTask(index)
// Tell the DAGScheduler that this task was resubmitted so that it doesn't think our
// stage finishes when a total of tasks.size tasks finish.
@@ -628,7 +612,7 @@ private[spark] class ClusterTaskSetManager(
}
// Also re-enqueue any tasks that were running on the node
for ((tid, info) <- taskInfos if info.running && info.executorId == execId) {
- taskLost(tid, TaskState.KILLED, null)
+ handleFailedTask(tid, TaskState.KILLED, None)
}
}
@@ -641,24 +625,24 @@ private[spark] class ClusterTaskSetManager(
*/
override def checkSpeculatableTasks(): Boolean = {
// Can't speculate if we only have one task, or if all tasks have finished.
- if (numTasks == 1 || tasksFinished == numTasks) {
+ if (numTasks == 1 || tasksSuccessful == numTasks) {
return false
}
var foundTasks = false
val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt
logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation)
- if (tasksFinished >= minFinishedForSpeculation) {
+ if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) {
val time = clock.getTime()
val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray
Arrays.sort(durations)
- val medianDuration = durations(min((0.5 * numTasks).round.toInt, durations.size - 1))
+ val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1))
val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100)
// TODO: Threshold should also look at standard deviation of task durations and have a lower
// bound based on that.
logDebug("Task length threshold for speculation: " + threshold)
for ((tid, info) <- taskInfos) {
val index = info.index
- if (!finished(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold &&
+ if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold &&
!speculatableTasks.contains(index)) {
logInfo(
"Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format(
@@ -672,7 +656,7 @@ private[spark] class ClusterTaskSetManager(
}
override def hasPendingTasks(): Boolean = {
- numTasks > 0 && tasksFinished < numTasks
+ numTasks > 0 && tasksSuccessful < numTasks
}
private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
index d57eb3276f..5367218faa 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
@@ -17,7 +17,7 @@
package org.apache.spark.scheduler.cluster
-import org.apache.spark.{SparkContext}
+import org.apache.spark.SparkContext
/**
* A backend interface for cluster scheduling systems that allows plugging in different ones under
@@ -30,8 +30,8 @@ private[spark] trait SchedulerBackend {
def reviveOffers(): Unit
def defaultParallelism(): Int
+ def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException
+
// Memory used by each executor (in megabytes)
protected val executorMemory: Int = SparkContext.executorMemoryRequested
-
- // TODO: Probably want to add a killTask too
}
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 d003bf1bba..cb88159b8d 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
@@ -26,7 +26,7 @@ import org.apache.spark.util.Utils
private[spark] class SparkDeploySchedulerBackend(
scheduler: ClusterScheduler,
sc: SparkContext,
- master: String,
+ masters: Array[String],
appName: String)
extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
with ClientListener
@@ -50,9 +50,9 @@ private[spark] class SparkDeploySchedulerBackend(
"org.apache.spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
val sparkHome = sc.getSparkHome().getOrElse(null)
val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
- sc.ui.appUIAddress)
+ "http://" + sc.ui.appUIAddress)
- client = new Client(sc.env.actorSystem, master, appDesc, this)
+ client = new Client(sc.env.actorSystem, masters, appDesc, this)
client.start()
}
@@ -71,22 +71,28 @@ private[spark] class SparkDeploySchedulerBackend(
override def disconnected() {
if (!stopping) {
- logError("Disconnected from Spark cluster!")
- scheduler.error("Disconnected from Spark cluster")
+ logWarning("Disconnected from Spark cluster! Waiting for reconnection...")
}
}
- override def executorAdded(executorId: String, workerId: String, hostPort: String, cores: Int, memory: Int) {
+ override def dead() {
+ if (!stopping) {
+ logError("Spark cluster looks dead, giving up.")
+ scheduler.error("Spark cluster looks down")
+ }
+ }
+
+ override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) {
logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format(
- executorId, hostPort, cores, Utils.megabytesToString(memory)))
+ fullId, hostPort, cores, Utils.megabytesToString(memory)))
}
- override def executorRemoved(executorId: String, message: String, exitStatus: Option[Int]) {
+ override def executorRemoved(fullId: String, message: String, exitStatus: Option[Int]) {
val reason: ExecutorLossReason = exitStatus match {
case Some(code) => ExecutorExited(code)
case None => SlaveLost(message)
}
- logInfo("Executor %s removed: %s".format(executorId, message))
- removeExecutor(executorId, reason.toString)
+ logInfo("Executor %s removed: %s".format(fullId, message))
+ removeExecutor(fullId.split("/")(1), reason.toString)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
index 9c36d221f6..12b2fd01c0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneClusterMessage.scala
@@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster
import java.nio.ByteBuffer
import org.apache.spark.TaskState.TaskState
+import org.apache.spark.scheduler.TaskDescription
import org.apache.spark.util.{Utils, SerializableBuffer}
@@ -30,6 +31,8 @@ private[spark] object StandaloneClusterMessages {
// Driver to executors
case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage
+ case class KillTask(taskId: Long, executor: String) extends StandaloneClusterMessage
+
case class RegisteredExecutor(sparkProperties: Seq[(String, String)])
extends StandaloneClusterMessage
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 b4ea0be415..08ee2182a2 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
@@ -29,6 +29,7 @@ import akka.util.Duration
import akka.util.duration._
import org.apache.spark.{SparkException, Logging, TaskState}
+import org.apache.spark.scheduler.TaskDescription
import org.apache.spark.scheduler.cluster.StandaloneClusterMessages._
import org.apache.spark.util.Utils
@@ -90,6 +91,9 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
case ReviveOffers =>
makeOffers()
+ case KillTask(taskId, executorId) =>
+ executorActor(executorId) ! KillTask(taskId, executorId)
+
case StopDriver =>
sender ! true
context.stop(self)
@@ -179,6 +183,10 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
driverActor ! ReviveOffers
}
+ override def killTask(taskId: Long, executorId: String) {
+ driverActor ! KillTask(taskId, executorId)
+ }
+
override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism"))
.map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2))
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
new file mode 100644
index 0000000000..feec8ecfe4
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import java.nio.ByteBuffer
+import java.util.concurrent.{LinkedBlockingDeque, ThreadFactory, ThreadPoolExecutor, TimeUnit}
+
+import org.apache.spark._
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult}
+import org.apache.spark.serializer.SerializerInstance
+
+/**
+ * Runs a thread pool that deserializes and remotely fetches (if necessary) task results.
+ */
+private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler)
+ extends Logging {
+ private val MIN_THREADS = System.getProperty("spark.resultGetter.minThreads", "4").toInt
+ private val MAX_THREADS = System.getProperty("spark.resultGetter.maxThreads", "4").toInt
+ private val getTaskResultExecutor = new ThreadPoolExecutor(
+ MIN_THREADS,
+ MAX_THREADS,
+ 0L,
+ TimeUnit.SECONDS,
+ new LinkedBlockingDeque[Runnable],
+ new ResultResolverThreadFactory)
+
+ class ResultResolverThreadFactory extends ThreadFactory {
+ private var counter = 0
+ private var PREFIX = "Result resolver thread"
+
+ override def newThread(r: Runnable): Thread = {
+ val thread = new Thread(r, "%s-%s".format(PREFIX, counter))
+ counter += 1
+ thread.setDaemon(true)
+ return thread
+ }
+ }
+
+ protected val serializer = new ThreadLocal[SerializerInstance] {
+ override def initialValue(): SerializerInstance = {
+ return sparkEnv.closureSerializer.newInstance()
+ }
+ }
+
+ def enqueueSuccessfulTask(
+ taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) {
+ getTaskResultExecutor.execute(new Runnable {
+ override def run() {
+ try {
+ val result = serializer.get().deserialize[TaskResult[_]](serializedData) match {
+ case directResult: DirectTaskResult[_] => directResult
+ case IndirectTaskResult(blockId) =>
+ logDebug("Fetching indirect task result for TID %s".format(tid))
+ val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId)
+ if (!serializedTaskResult.isDefined) {
+ /* We won't be able to get the task result if the machine that ran the task failed
+ * between when the task ended and when we tried to fetch the result, or if the
+ * block manager had to flush the result. */
+ scheduler.handleFailedTask(
+ taskSetManager, tid, TaskState.FINISHED, Some(TaskResultLost))
+ return
+ }
+ val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]](
+ serializedTaskResult.get)
+ sparkEnv.blockManager.master.removeBlock(blockId)
+ deserializedResult
+ }
+ result.metrics.resultSize = serializedData.limit()
+ scheduler.handleSuccessfulTask(taskSetManager, tid, result)
+ } catch {
+ case cnf: ClassNotFoundException =>
+ val loader = Thread.currentThread.getContextClassLoader
+ taskSetManager.abort("ClassNotFound with classloader: " + loader)
+ case ex =>
+ taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex))
+ }
+ }
+ })
+ }
+
+ def enqueueFailedTask(taskSetManager: ClusterTaskSetManager, tid: Long, taskState: TaskState,
+ serializedData: ByteBuffer) {
+ var reason: Option[TaskEndReason] = None
+ getTaskResultExecutor.execute(new Runnable {
+ override def run() {
+ try {
+ if (serializedData != null && serializedData.limit() > 0) {
+ reason = Some(serializer.get().deserialize[TaskEndReason](
+ serializedData, getClass.getClassLoader))
+ }
+ } catch {
+ case cnd: ClassNotFoundException =>
+ // Log an error but keep going here -- the task failed, so not catastropic if we can't
+ // deserialize the reason.
+ val loader = Thread.currentThread.getContextClassLoader
+ logError(
+ "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader)
+ case ex => {}
+ }
+ scheduler.handleFailedTask(taskSetManager, tid, taskState, reason)
+ }
+ })
+ }
+
+ def stop() {
+ getTaskResultExecutor.shutdownNow()
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 3dbe61d706..8f2eef9a53 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -15,22 +15,22 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.mesos
+package org.apache.spark.scheduler.cluster.mesos
-import com.google.protobuf.ByteString
+import java.io.File
+import java.util.{ArrayList => JArrayList, List => JList}
+import java.util.Collections
+
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.collection.JavaConversions._
+import com.google.protobuf.ByteString
import org.apache.mesos.{Scheduler => MScheduler}
import org.apache.mesos._
import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
-import org.apache.spark.{SparkException, Logging, SparkContext}
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import scala.collection.JavaConversions._
-import java.io.File
-import org.apache.spark.scheduler.cluster._
-import java.util.{ArrayList => JArrayList, List => JList}
-import java.util.Collections
-import org.apache.spark.TaskState
+import org.apache.spark.{SparkException, Logging, SparkContext, TaskState}
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, StandaloneSchedulerBackend}
/**
* A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
diff --git a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 541f86e338..50cbc2ca92 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -15,22 +15,24 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.mesos
+package org.apache.spark.scheduler.cluster.mesos
-import com.google.protobuf.ByteString
+import java.io.File
+import java.util.{ArrayList => JArrayList, List => JList}
+import java.util.Collections
+
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.collection.JavaConversions._
+import com.google.protobuf.ByteString
import org.apache.mesos.{Scheduler => MScheduler}
import org.apache.mesos._
import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
-import org.apache.spark.{SparkException, Logging, SparkContext}
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
-import scala.collection.JavaConversions._
-import java.io.File
-import org.apache.spark.scheduler.cluster._
-import java.util.{ArrayList => JArrayList, List => JList}
-import java.util.Collections
-import org.apache.spark.TaskState
+import org.apache.spark.{Logging, SparkException, SparkContext, TaskState}
+import org.apache.spark.scheduler.TaskDescription
+import org.apache.spark.scheduler.cluster.{ClusterScheduler, ExecutorExited, ExecutorLossReason}
+import org.apache.spark.scheduler.cluster.{SchedulerBackend, SlaveLost, WorkerOffer}
import org.apache.spark.util.Utils
/**
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
index e8fa5e2f17..b445260d1b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
@@ -17,24 +17,19 @@
package org.apache.spark.scheduler.local
-import java.io.File
-import java.lang.management.ManagementFactory
-import java.util.concurrent.atomic.AtomicInteger
import java.nio.ByteBuffer
+import java.util.concurrent.atomic.AtomicInteger
-import scala.collection.JavaConversions._
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
+import akka.actor._
import org.apache.spark._
import org.apache.spark.TaskState.TaskState
-import org.apache.spark.executor.ExecutorURLClassLoader
+import org.apache.spark.executor.{Executor, ExecutorBackend}
import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster._
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
-import akka.actor._
-import org.apache.spark.util.Utils
+import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
+
/**
* A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally
@@ -42,43 +37,50 @@ import org.apache.spark.util.Utils
* testing fault recovery.
*/
-private[spark]
+private[local]
case class LocalReviveOffers()
-private[spark]
+private[local]
case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer)
+private[local]
+case class KillTask(taskId: Long)
+
private[spark]
-class LocalActor(localScheduler: LocalScheduler, var freeCores: Int) extends Actor with Logging {
+class LocalActor(localScheduler: LocalScheduler, private var freeCores: Int)
+ extends Actor with Logging {
+
+ val executor = new Executor("localhost", "localhost", Seq.empty, isLocal = true)
def receive = {
case LocalReviveOffers =>
launchTask(localScheduler.resourceOffer(freeCores))
+
case LocalStatusUpdate(taskId, state, serializeData) =>
- freeCores += 1
- localScheduler.statusUpdate(taskId, state, serializeData)
- launchTask(localScheduler.resourceOffer(freeCores))
+ if (TaskState.isFinished(state)) {
+ freeCores += 1
+ launchTask(localScheduler.resourceOffer(freeCores))
+ }
+
+ case KillTask(taskId) =>
+ executor.killTask(taskId)
}
- def launchTask(tasks : Seq[TaskDescription]) {
+ private def launchTask(tasks: Seq[TaskDescription]) {
for (task <- tasks) {
freeCores -= 1
- localScheduler.threadPool.submit(new Runnable {
- def run() {
- localScheduler.runTask(task.taskId, task.serializedTask)
- }
- })
+ executor.launchTask(localScheduler, task.taskId, task.serializedTask)
}
}
}
private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc: SparkContext)
extends TaskScheduler
+ with ExecutorBackend
with Logging {
- var attemptId = new AtomicInteger(0)
- var threadPool = Utils.newDaemonFixedThreadPool(threads)
val env = SparkEnv.get
+ val attemptId = new AtomicInteger
var listener: TaskSchedulerListener = null
// Application dependencies (added through SparkContext) that we've fetched so far on this node.
@@ -86,13 +88,11 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
val currentFiles: HashMap[String, Long] = new HashMap[String, Long]()
val currentJars: HashMap[String, Long] = new HashMap[String, Long]()
- val classLoader = new ExecutorURLClassLoader(Array(), Thread.currentThread.getContextClassLoader)
-
var schedulableBuilder: SchedulableBuilder = null
var rootPool: Pool = null
val schedulingMode: SchedulingMode = SchedulingMode.withName(
- System.getProperty("spark.cluster.schedulingmode", "FIFO"))
- val activeTaskSets = new HashMap[String, TaskSetManager]
+ System.getProperty("spark.scheduler.mode", "FIFO"))
+ val activeTaskSets = new HashMap[String, LocalTaskSetManager]
val taskIdToTaskSetId = new HashMap[Long, String]
val taskSetTaskIds = new HashMap[String, HashSet[Long]]
@@ -128,6 +128,26 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
}
}
+ override def cancelTasks(stageId: Int): Unit = synchronized {
+ logInfo("Cancelling stage " + stageId)
+ logInfo("Cancelling stage " + activeTaskSets.map(_._2.stageId))
+ activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) =>
+ // There are two possible cases here:
+ // 1. The task set manager has been created and some tasks have been scheduled.
+ // In this case, send a kill signal to the executors to kill the task and then abort
+ // the stage.
+ // 2. The task set manager has been created but no tasks has been scheduled. In this case,
+ // simply abort the stage.
+ val taskIds = taskSetTaskIds(tsm.taskSet.id)
+ if (taskIds.size > 0) {
+ taskIds.foreach { tid =>
+ localActor ! KillTask(tid)
+ }
+ }
+ tsm.error("Stage %d was cancelled".format(stageId))
+ }
+ }
+
def resourceOffer(freeCores: Int): Seq[TaskDescription] = {
synchronized {
var freeCpuCores = freeCores
@@ -167,106 +187,32 @@ private[spark] class LocalScheduler(threads: Int, val maxFailures: Int, val sc:
}
}
- def runTask(taskId: Long, bytes: ByteBuffer) {
- logInfo("Running " + taskId)
- val info = new TaskInfo(taskId, 0, System.currentTimeMillis(), "local", "local:1", TaskLocality.NODE_LOCAL)
- // Set the Spark execution environment for the worker thread
- SparkEnv.set(env)
- val ser = SparkEnv.get.closureSerializer.newInstance()
- val objectSer = SparkEnv.get.serializer.newInstance()
- var attemptedTask: Option[Task[_]] = None
- val start = System.currentTimeMillis()
- var taskStart: Long = 0
- def getTotalGCTime = ManagementFactory.getGarbageCollectorMXBeans.map(g => g.getCollectionTime).sum
- val startGCTime = getTotalGCTime
-
- try {
- Accumulators.clear()
- Thread.currentThread().setContextClassLoader(classLoader)
-
- // Serialize and deserialize the task so that accumulators are changed to thread-local ones;
- // this adds a bit of unnecessary overhead but matches how the Mesos Executor works.
- val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes)
- updateDependencies(taskFiles, taskJars) // Download any files added with addFile
- val deserializedTask = ser.deserialize[Task[_]](
- taskBytes, Thread.currentThread.getContextClassLoader)
- attemptedTask = Some(deserializedTask)
- val deserTime = System.currentTimeMillis() - start
- taskStart = System.currentTimeMillis()
-
- // Run it
- val result: Any = deserializedTask.run(taskId)
-
- // Serialize and deserialize the result to emulate what the Mesos
- // executor does. This is useful to catch serialization errors early
- // on in development (so when users move their local Spark programs
- // to the cluster, they don't get surprised by serialization errors).
- val serResult = objectSer.serialize(result)
- deserializedTask.metrics.get.resultSize = serResult.limit()
- val resultToReturn = objectSer.deserialize[Any](serResult)
- val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]](
- ser.serialize(Accumulators.values))
- val serviceTime = System.currentTimeMillis() - taskStart
- logInfo("Finished " + taskId)
- deserializedTask.metrics.get.executorRunTime = serviceTime.toInt
- deserializedTask.metrics.get.jvmGCTime = getTotalGCTime - startGCTime
- deserializedTask.metrics.get.executorDeserializeTime = deserTime.toInt
- val taskResult = new TaskResult(result, accumUpdates, deserializedTask.metrics.getOrElse(null))
- val serializedResult = ser.serialize(taskResult)
- localActor ! LocalStatusUpdate(taskId, TaskState.FINISHED, serializedResult)
- } catch {
- case t: Throwable => {
- val serviceTime = System.currentTimeMillis() - taskStart
- val metrics = attemptedTask.flatMap(t => t.metrics)
- for (m <- metrics) {
- m.executorRunTime = serviceTime.toInt
- m.jvmGCTime = getTotalGCTime - startGCTime
- }
- val failure = new ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics)
- localActor ! LocalStatusUpdate(taskId, TaskState.FAILED, ser.serialize(failure))
- }
- }
- }
-
- /**
- * Download any missing dependencies if we receive a new set of files and JARs from the
- * SparkContext. Also adds any new JARs we fetched to the class loader.
- */
- private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) {
- synchronized {
- // Fetch missing dependencies
- for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
- logInfo("Fetching " + name + " with timestamp " + timestamp)
- Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
- currentFiles(name) = timestamp
- }
-
- for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
- logInfo("Fetching " + name + " with timestamp " + timestamp)
- Utils.fetchFile(name, new File(SparkFiles.getRootDirectory))
- currentJars(name) = timestamp
- // Add it to our class loader
- val localName = name.split("/").last
- val url = new File(SparkFiles.getRootDirectory, localName).toURI.toURL
- if (!classLoader.getURLs.contains(url)) {
- logInfo("Adding " + url + " to class loader")
- classLoader.addURL(url)
+ override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) {
+ if (TaskState.isFinished(state)) {
+ synchronized {
+ taskIdToTaskSetId.get(taskId) match {
+ case Some(taskSetId) =>
+ val taskSetManager = activeTaskSets(taskSetId)
+ taskSetTaskIds(taskSetId) -= taskId
+
+ state match {
+ case TaskState.FINISHED =>
+ taskSetManager.taskEnded(taskId, state, serializedData)
+ case TaskState.FAILED =>
+ taskSetManager.taskFailed(taskId, state, serializedData)
+ case TaskState.KILLED =>
+ taskSetManager.error("Task %d was killed".format(taskId))
+ case _ => {}
+ }
+ case None =>
+ logInfo("Ignoring update from TID " + taskId + " because its task set is gone")
}
}
+ localActor ! LocalStatusUpdate(taskId, state, serializedData)
}
}
- def statusUpdate(taskId :Long, state: TaskState, serializedData: ByteBuffer) {
- synchronized {
- val taskSetId = taskIdToTaskSetId(taskId)
- val taskSetManager = activeTaskSets(taskSetId)
- taskSetTaskIds(taskSetId) -= taskId
- taskSetManager.statusUpdate(taskId, state, serializedData)
- }
- }
-
- override def stop() {
- threadPool.shutdownNow()
+ override def stop() {
}
override def defaultParallelism() = threads
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
index e52cb998bd..f72e77d40f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
@@ -21,16 +21,16 @@ import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
-import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, Success, TaskState}
+import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, SparkException, Success, TaskState}
import org.apache.spark.TaskState.TaskState
-import org.apache.spark.scheduler.{Task, TaskResult, TaskSet}
-import org.apache.spark.scheduler.cluster.{Schedulable, TaskDescription, TaskInfo, TaskLocality, TaskSetManager}
+import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Pool, Schedulable, Task,
+ TaskDescription, TaskInfo, TaskLocality, TaskResult, TaskSet, TaskSetManager}
private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: TaskSet)
extends TaskSetManager with Logging {
- var parent: Schedulable = null
+ var parent: Pool = null
var weight: Int = 1
var minShare: Int = 0
var runningTasks: Int = 0
@@ -49,14 +49,14 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
val numFailures = new Array[Int](numTasks)
val MAX_TASK_FAILURES = sched.maxFailures
- override def increaseRunningTasks(taskNum: Int): Unit = {
+ def increaseRunningTasks(taskNum: Int): Unit = {
runningTasks += taskNum
if (parent != null) {
parent.increaseRunningTasks(taskNum)
}
}
- override def decreaseRunningTasks(taskNum: Int): Unit = {
+ def decreaseRunningTasks(taskNum: Int): Unit = {
runningTasks -= taskNum
if (parent != null) {
parent.decreaseRunningTasks(taskNum)
@@ -132,17 +132,6 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
return None
}
- override def statusUpdate(tid: Long, state: TaskState, serializedData: ByteBuffer) {
- SparkEnv.set(env)
- state match {
- case TaskState.FINISHED =>
- taskEnded(tid, state, serializedData)
- case TaskState.FAILED =>
- taskFailed(tid, state, serializedData)
- case _ => {}
- }
- }
-
def taskStarted(task: Task[_], info: TaskInfo) {
sched.listener.taskStarted(task, info)
}
@@ -152,7 +141,12 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
val index = info.index
val task = taskSet.tasks(index)
info.markSuccessful()
- val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader)
+ val result = ser.deserialize[TaskResult[_]](serializedData, getClass.getClassLoader) match {
+ case directResult: DirectTaskResult[_] => directResult
+ case IndirectTaskResult(blockId) => {
+ throw new SparkException("Expect only DirectTaskResults when using LocalScheduler")
+ }
+ }
result.metrics.resultSize = serializedData.limit()
sched.listener.taskEnded(task, Success, result.value, result.accumUpdates, info, result.metrics)
numFinished += 1
@@ -190,5 +184,7 @@ private[spark] class LocalTaskSetManager(sched: LocalScheduler, val taskSet: Tas
}
override def error(message: String) {
+ sched.listener.taskSetFailed(taskSet, message)
+ sched.taskSetFinished(this)
}
}
diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
index 24ef204aa1..55b25f145a 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -23,12 +23,11 @@ import java.io.{EOFException, InputStream, OutputStream}
import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer}
import com.esotericsoftware.kryo.{KryoException, Kryo}
import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
-import com.twitter.chill.ScalaKryoInstantiator
+import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar}
import org.apache.spark.{SerializableWritable, Logging}
-import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock, StorageLevel}
-
import org.apache.spark.broadcast.HttpBroadcast
+import org.apache.spark.storage.{GetBlock,GotBlock, PutBlock, StorageLevel, TestBlockId}
/**
* A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]].
@@ -38,20 +37,23 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
def newKryoOutput() = new KryoOutput(bufferSize)
- def newKryoInput() = new KryoInput(bufferSize)
-
def newKryo(): Kryo = {
- val instantiator = new ScalaKryoInstantiator
+ val instantiator = new EmptyScalaKryoInstantiator
val kryo = instantiator.newKryo()
val classLoader = Thread.currentThread.getContextClassLoader
+ val blockId = TestBlockId("1")
// Register some commonly used classes
val toRegister: Seq[AnyRef] = Seq(
ByteBuffer.allocate(1),
StorageLevel.MEMORY_ONLY,
- PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY),
- GotBlock("1", ByteBuffer.allocate(1)),
- GetBlock("1")
+ PutBlock(blockId, ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY),
+ GotBlock(blockId, ByteBuffer.allocate(1)),
+ GetBlock(blockId),
+ 1 to 10,
+ 1 until 10,
+ 1L to 10L,
+ 1L until 10L
)
for (obj <- toRegister) kryo.register(obj.getClass)
@@ -71,6 +73,10 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
case _: Exception => println("Failed to register spark.kryo.registrator")
}
+ // Register Chill's classes; we do this after our ranges and the user's own classes to let
+ // our code override the generic serialziers in Chill for things like Seq
+ new AllScalaRegistrar().apply(kryo)
+
kryo.setClassLoader(classLoader)
// Allow disabling Kryo reference tracking if user knows their object graphs don't have loops
@@ -118,8 +124,10 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser
private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance {
val kryo = ks.newKryo()
- val output = ks.newKryoOutput()
- val input = ks.newKryoInput()
+
+ // Make these lazy vals to avoid creating a buffer unless we use them
+ lazy val output = ks.newKryoOutput()
+ lazy val input = new KryoInput()
def serialize[T](t: T): ByteBuffer = {
output.clear()
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockException.scala b/core/src/main/scala/org/apache/spark/storage/BlockException.scala
index 290dbce4f5..0d0a2dadc7 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockException.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockException.scala
@@ -18,5 +18,5 @@
package org.apache.spark.storage
private[spark]
-case class BlockException(blockId: String, message: String) extends Exception(message)
+case class BlockException(blockId: BlockId, message: String) extends Exception(message)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
index 3aeda3879d..e51c5b30a3 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
@@ -47,7 +47,7 @@ import org.apache.spark.util.Utils
*/
private[storage]
-trait BlockFetcherIterator extends Iterator[(String, Option[Iterator[Any]])]
+trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])]
with Logging with BlockFetchTracker {
def initialize()
}
@@ -57,20 +57,20 @@ private[storage]
object BlockFetcherIterator {
// A request to fetch one or more blocks, complete with their sizes
- class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) {
+ class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) {
val size = blocks.map(_._2).sum
}
// A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize
// the block (since we want all deserializaton to happen in the calling thread); can also
// represent a fetch failure if size == -1.
- class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) {
+ class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) {
def failed: Boolean = size == -1
}
class BasicBlockFetcherIterator(
private val blockManager: BlockManager,
- val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])],
+ val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
serializer: Serializer)
extends BlockFetcherIterator {
@@ -92,12 +92,12 @@ object BlockFetcherIterator {
// This represents the number of local blocks, also counting zero-sized blocks
private var numLocal = 0
// BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks
- protected val localBlocksToFetch = new ArrayBuffer[String]()
+ protected val localBlocksToFetch = new ArrayBuffer[BlockId]()
// This represents the number of remote blocks, also counting zero-sized blocks
private var numRemote = 0
// BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks
- protected val remoteBlocksToFetch = new HashSet[String]()
+ protected val remoteBlocksToFetch = new HashSet[BlockId]()
// A queue to hold our results.
protected val results = new LinkedBlockingQueue[FetchResult]
@@ -167,7 +167,7 @@ object BlockFetcherIterator {
logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize)
val iterator = blockInfos.iterator
var curRequestSize = 0L
- var curBlocks = new ArrayBuffer[(String, Long)]
+ var curBlocks = new ArrayBuffer[(BlockId, Long)]
while (iterator.hasNext) {
val (blockId, size) = iterator.next()
// Skip empty blocks
@@ -183,7 +183,7 @@ object BlockFetcherIterator {
// Add this FetchRequest
remoteRequests += new FetchRequest(address, curBlocks)
curRequestSize = 0
- curBlocks = new ArrayBuffer[(String, Long)]
+ curBlocks = new ArrayBuffer[(BlockId, Long)]
}
}
// Add in the final request
@@ -241,7 +241,7 @@ object BlockFetcherIterator {
override def hasNext: Boolean = resultsGotten < _numBlocksToFetch
- override def next(): (String, Option[Iterator[Any]]) = {
+ override def next(): (BlockId, Option[Iterator[Any]]) = {
resultsGotten += 1
val startFetchWait = System.currentTimeMillis()
val result = results.take()
@@ -267,7 +267,7 @@ object BlockFetcherIterator {
class NettyBlockFetcherIterator(
blockManager: BlockManager,
- blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])],
+ blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
serializer: Serializer)
extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer) {
@@ -303,7 +303,7 @@ object BlockFetcherIterator {
override protected def sendRequest(req: FetchRequest) {
- def putResult(blockId: String, blockSize: Long, blockData: ByteBuf) {
+ def putResult(blockId: BlockId, blockSize: Long, blockData: ByteBuf) {
val fetchResult = new FetchResult(blockId, blockSize,
() => dataDeserialize(blockId, blockData.nioBuffer, serializer))
results.put(fetchResult)
@@ -337,7 +337,7 @@ object BlockFetcherIterator {
logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")
}
- override def next(): (String, Option[Iterator[Any]]) = {
+ override def next(): (BlockId, Option[Iterator[Any]]) = {
resultsGotten += 1
val result = results.take()
// If all the results has been retrieved, copiers will exit automatically
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
new file mode 100644
index 0000000000..c7efc67a4a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
@@ -0,0 +1,96 @@
+/*
+ * 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
+
+/**
+ * Identifies a particular Block of data, usually associated with a single file.
+ * A Block can be uniquely identified by its filename, but each type of Block has a different
+ * set of keys which produce its unique name.
+ *
+ * If your BlockId should be serializable, be sure to add it to the BlockId.fromString() method.
+ */
+private[spark] sealed abstract class BlockId {
+ /** A globally unique identifier for this Block. Can be used for ser/de. */
+ def name: String
+
+ // convenience methods
+ def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None
+ def isRDD = isInstanceOf[RDDBlockId]
+ def isShuffle = isInstanceOf[ShuffleBlockId]
+ def isBroadcast = isInstanceOf[BroadcastBlockId]
+
+ override def toString = name
+ override def hashCode = name.hashCode
+ override def equals(other: Any): Boolean = other match {
+ case o: BlockId => getClass == o.getClass && name.equals(o.name)
+ case _ => false
+ }
+}
+
+private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId {
+ def name = "rdd_" + rddId + "_" + splitIndex
+}
+
+private[spark]
+case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId {
+ def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId
+}
+
+private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId {
+ def name = "broadcast_" + broadcastId
+}
+
+private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId {
+ def name = "taskresult_" + taskId
+}
+
+private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId {
+ def name = "input-" + streamId + "-" + uniqueId
+}
+
+// Intended only for testing purposes
+private[spark] case class TestBlockId(id: String) extends BlockId {
+ def name = "test_" + id
+}
+
+private[spark] object BlockId {
+ val RDD = "rdd_([0-9]+)_([0-9]+)".r
+ val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r
+ val BROADCAST = "broadcast_([0-9]+)".r
+ val TASKRESULT = "taskresult_([0-9]+)".r
+ val STREAM = "input-([0-9]+)-([0-9]+)".r
+ val TEST = "test_(.*)".r
+
+ /** Converts a BlockId "name" String back into a BlockId. */
+ def apply(id: String) = id match {
+ case RDD(rddId, splitIndex) =>
+ RDDBlockId(rddId.toInt, splitIndex.toInt)
+ case SHUFFLE(shuffleId, mapId, reduceId) =>
+ ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt)
+ case BROADCAST(broadcastId) =>
+ BroadcastBlockId(broadcastId.toLong)
+ case TASKRESULT(taskId) =>
+ TaskResultBlockId(taskId.toLong)
+ case STREAM(streamId, uniqueId) =>
+ StreamBlockId(streamId.toInt, uniqueId.toLong)
+ case TEST(value) =>
+ TestBlockId(value)
+ case _ =>
+ throw new IllegalStateException("Unrecognized BlockId: " + id)
+ }
+}
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 60fdc5f2ee..801f88a3db 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -37,7 +37,6 @@ import org.apache.spark.util._
import sun.nio.ch.DirectBuffer
-
private[spark] class BlockManager(
executorId: String,
actorSystem: ActorSystem,
@@ -103,7 +102,7 @@ private[spark] class BlockManager(
val shuffleBlockManager = new ShuffleBlockManager(this)
- private val blockInfo = new TimeStampedHashMap[String, BlockInfo]
+ private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo]
private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory)
private[storage] val diskStore: DiskStore =
@@ -154,7 +153,8 @@ private[spark] class BlockManager(
var heartBeatTask: Cancellable = null
- val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks)
+ private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks)
+ private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks)
initialize()
// The compression codec to use. Note that the "lazy" val is necessary because we want to delay
@@ -248,7 +248,7 @@ private[spark] class BlockManager(
/**
* Get storage level of local block. If no info exists for the block, then returns null.
*/
- def getLevel(blockId: String): StorageLevel = blockInfo.get(blockId).map(_.level).orNull
+ def getLevel(blockId: BlockId): StorageLevel = blockInfo.get(blockId).map(_.level).orNull
/**
* Tell the master about the current storage status of a block. This will send a block update
@@ -258,7 +258,7 @@ private[spark] class BlockManager(
* droppedMemorySize exists to account for when block is dropped from memory to disk (so it is still valid).
* This ensures that update in master will compensate for the increase in memory on slave.
*/
- def reportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L) {
+ def reportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L) {
val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize)
if (needReregister) {
logInfo("Got told to reregister updating block " + blockId)
@@ -273,7 +273,7 @@ private[spark] class BlockManager(
* which will be true if the block was successfully recorded and false if
* the slave needs to re-register.
*/
- private def tryToReportBlockStatus(blockId: String, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = {
+ private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L): Boolean = {
val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized {
info.level match {
case null =>
@@ -298,7 +298,7 @@ private[spark] class BlockManager(
/**
* Get locations of an array of blocks.
*/
- def getLocationBlockIds(blockIds: Array[String]): Array[Seq[BlockManagerId]] = {
+ def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = {
val startTimeMs = System.currentTimeMillis
val locations = master.getLocations(blockIds).toArray
logDebug("Got multiple block location in " + Utils.getUsedTimeMs(startTimeMs))
@@ -310,7 +310,7 @@ private[spark] class BlockManager(
* shuffle blocks. It is safe to do so without a lock on block info since disk store
* never deletes (recent) items.
*/
- def getLocalFromDisk(blockId: String, serializer: Serializer): Option[Iterator[Any]] = {
+ def getLocalFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = {
diskStore.getValues(blockId, serializer).orElse(
sys.error("Block " + blockId + " not found on disk, though it should be"))
}
@@ -318,7 +318,7 @@ private[spark] class BlockManager(
/**
* Get block from local block manager.
*/
- def getLocal(blockId: String): Option[Iterator[Any]] = {
+ def getLocal(blockId: BlockId): Option[Iterator[Any]] = {
logDebug("Getting local block " + blockId)
val info = blockInfo.get(blockId).orNull
if (info != null) {
@@ -399,13 +399,13 @@ private[spark] class BlockManager(
/**
* Get block from the local block manager as serialized bytes.
*/
- def getLocalBytes(blockId: String): Option[ByteBuffer] = {
+ def getLocalBytes(blockId: BlockId): Option[ByteBuffer] = {
// TODO: This whole thing is very similar to getLocal; we need to refactor it somehow
logDebug("Getting local block " + blockId + " as bytes")
// As an optimization for map output fetches, if the block is for a shuffle, return it
// without acquiring a lock; the disk store never deletes (recent) items so this should work
- if (ShuffleBlockManager.isShuffle(blockId)) {
+ if (blockId.isShuffle) {
return diskStore.getBytes(blockId) match {
case Some(bytes) =>
Some(bytes)
@@ -472,7 +472,7 @@ private[spark] class BlockManager(
/**
* Get block from remote block managers.
*/
- def getRemote(blockId: String): Option[Iterator[Any]] = {
+ def getRemote(blockId: BlockId): Option[Iterator[Any]] = {
if (blockId == null) {
throw new IllegalArgumentException("Block Id is null")
}
@@ -484,7 +484,7 @@ private[spark] class BlockManager(
for (loc <- locations) {
logDebug("Getting remote block " + blockId + " from " + loc)
val data = BlockManagerWorker.syncGetBlock(
- GetBlock(blockId), ConnectionManagerId(loc.host, loc.port))
+ GetBlock(blockId), ConnectionManagerId(loc.host, loc.port))
if (data != null) {
return Some(dataDeserialize(blockId, data))
}
@@ -495,10 +495,45 @@ private[spark] class BlockManager(
}
/**
+ * Get block from remote block managers as serialized bytes.
+ */
+ def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = {
+ // TODO: As with getLocalBytes, this is very similar to getRemote and perhaps should be
+ // refactored.
+ if (blockId == null) {
+ throw new IllegalArgumentException("Block Id is null")
+ }
+ logDebug("Getting remote block " + blockId + " as bytes")
+
+ val locations = master.getLocations(blockId)
+ for (loc <- locations) {
+ logDebug("Getting remote block " + blockId + " from " + loc)
+ val data = BlockManagerWorker.syncGetBlock(
+ GetBlock(blockId), ConnectionManagerId(loc.host, loc.port))
+ if (data != null) {
+ return Some(data)
+ }
+ logDebug("The value of block " + blockId + " is null")
+ }
+ logDebug("Block " + blockId + " not found")
+ return None
+ }
+
+ /**
* Get a block from the block manager (either local or remote).
*/
- def get(blockId: String): Option[Iterator[Any]] = {
- getLocal(blockId).orElse(getRemote(blockId))
+ def get(blockId: BlockId): Option[Iterator[Any]] = {
+ val local = getLocal(blockId)
+ if (local.isDefined) {
+ logInfo("Found block %s locally".format(blockId))
+ return local
+ }
+ val remote = getRemote(blockId)
+ if (remote.isDefined) {
+ logInfo("Found block %s remotely".format(blockId))
+ return remote
+ }
+ None
}
/**
@@ -508,7 +543,7 @@ private[spark] class BlockManager(
* so that we can control the maxMegabytesInFlight for the fetch.
*/
def getMultiple(
- blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])], serializer: Serializer)
+ blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer)
: BlockFetcherIterator = {
val iter =
@@ -522,7 +557,7 @@ private[spark] class BlockManager(
iter
}
- def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean)
+ def put(blockId: BlockId, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean)
: Long = {
val elements = new ArrayBuffer[Any]
elements ++= values
@@ -534,7 +569,7 @@ private[spark] class BlockManager(
* This is currently used for writing shuffle files out. Callers should handle error
* cases.
*/
- def getDiskBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int)
+ def getDiskBlockWriter(blockId: BlockId, serializer: Serializer, bufferSize: Int)
: BlockObjectWriter = {
val writer = diskStore.getBlockWriter(blockId, serializer, bufferSize)
writer.registerCloseEventHandler(() => {
@@ -548,7 +583,7 @@ private[spark] class BlockManager(
/**
* Put a new block of values to the block manager. Returns its (estimated) size in bytes.
*/
- def put(blockId: String, values: ArrayBuffer[Any], level: StorageLevel,
+ def put(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel,
tellMaster: Boolean = true) : Long = {
if (blockId == null) {
@@ -668,7 +703,7 @@ private[spark] class BlockManager(
* Put a new block of serialized bytes to the block manager.
*/
def putBytes(
- blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) {
+ blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) {
if (blockId == null) {
throw new IllegalArgumentException("Block Id is null")
@@ -769,7 +804,7 @@ private[spark] class BlockManager(
* Replicate block to another node.
*/
var cachedPeers: Seq[BlockManagerId] = null
- private def replicate(blockId: String, data: ByteBuffer, level: StorageLevel) {
+ private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) {
val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1)
if (cachedPeers == null) {
cachedPeers = master.getPeers(blockManagerId, level.replication - 1)
@@ -792,14 +827,14 @@ private[spark] class BlockManager(
/**
* Read a block consisting of a single object.
*/
- def getSingle(blockId: String): Option[Any] = {
+ def getSingle(blockId: BlockId): Option[Any] = {
get(blockId).map(_.next())
}
/**
* Write a block consisting of a single object.
*/
- def putSingle(blockId: String, value: Any, level: StorageLevel, tellMaster: Boolean = true) {
+ def putSingle(blockId: BlockId, value: Any, level: StorageLevel, tellMaster: Boolean = true) {
put(blockId, Iterator(value), level, tellMaster)
}
@@ -807,7 +842,7 @@ private[spark] class BlockManager(
* Drop a block from memory, possibly putting it on disk if applicable. Called when the memory
* store reaches its limit and needs to free up space.
*/
- def dropFromMemory(blockId: String, data: Either[ArrayBuffer[Any], ByteBuffer]) {
+ def dropFromMemory(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer]) {
logInfo("Dropping block " + blockId + " from memory")
val info = blockInfo.get(blockId).orNull
if (info != null) {
@@ -856,16 +891,15 @@ private[spark] class BlockManager(
// TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps
// from RDD.id to blocks.
logInfo("Removing RDD " + rddId)
- val rddPrefix = "rdd_" + rddId + "_"
- val blocksToRemove = blockInfo.filter(_._1.startsWith(rddPrefix)).map(_._1)
- blocksToRemove.foreach(blockId => removeBlock(blockId, false))
+ val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId)
+ blocksToRemove.foreach(blockId => removeBlock(blockId, tellMaster = false))
blocksToRemove.size
}
/**
* Remove a block from both memory and disk.
*/
- def removeBlock(blockId: String, tellMaster: Boolean = true) {
+ def removeBlock(blockId: BlockId, tellMaster: Boolean = true) {
logInfo("Removing block " + blockId)
val info = blockInfo.get(blockId).orNull
if (info != null) info.synchronized {
@@ -886,13 +920,13 @@ private[spark] class BlockManager(
}
}
- def dropOldBlocks(cleanupTime: Long) {
- logInfo("Dropping blocks older than " + cleanupTime)
+ private def dropOldNonBroadcastBlocks(cleanupTime: Long) {
+ logInfo("Dropping non broadcast blocks older than " + cleanupTime)
val iterator = blockInfo.internalMap.entrySet().iterator()
while (iterator.hasNext) {
val entry = iterator.next()
val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2)
- if (time < cleanupTime) {
+ if (time < cleanupTime && !id.isBroadcast) {
info.synchronized {
val level = info.level
if (level.useMemory) {
@@ -909,34 +943,52 @@ private[spark] class BlockManager(
}
}
- def shouldCompress(blockId: String): Boolean = {
- if (ShuffleBlockManager.isShuffle(blockId)) {
- compressShuffle
- } else if (blockId.startsWith("broadcast_")) {
- compressBroadcast
- } else if (blockId.startsWith("rdd_")) {
- compressRdds
- } else {
- false // Won't happen in a real cluster, but it can in tests
+ private def dropOldBroadcastBlocks(cleanupTime: Long) {
+ logInfo("Dropping broadcast blocks older than " + cleanupTime)
+ val iterator = blockInfo.internalMap.entrySet().iterator()
+ while (iterator.hasNext) {
+ val entry = iterator.next()
+ val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2)
+ if (time < cleanupTime && id.isBroadcast) {
+ info.synchronized {
+ val level = info.level
+ if (level.useMemory) {
+ memoryStore.remove(id)
+ }
+ if (level.useDisk) {
+ diskStore.remove(id)
+ }
+ iterator.remove()
+ logInfo("Dropped block " + id)
+ }
+ reportBlockStatus(id, info)
+ }
}
}
+ def shouldCompress(blockId: BlockId): Boolean = blockId match {
+ case ShuffleBlockId(_, _, _) => compressShuffle
+ case BroadcastBlockId(_) => compressBroadcast
+ case RDDBlockId(_, _) => compressRdds
+ case _ => false
+ }
+
/**
* Wrap an output stream for compression if block compression is enabled for its block type
*/
- def wrapForCompression(blockId: String, s: OutputStream): OutputStream = {
+ def wrapForCompression(blockId: BlockId, s: OutputStream): OutputStream = {
if (shouldCompress(blockId)) compressionCodec.compressedOutputStream(s) else s
}
/**
* Wrap an input stream for compression if block compression is enabled for its block type
*/
- def wrapForCompression(blockId: String, s: InputStream): InputStream = {
+ def wrapForCompression(blockId: BlockId, s: InputStream): InputStream = {
if (shouldCompress(blockId)) compressionCodec.compressedInputStream(s) else s
}
def dataSerialize(
- blockId: String,
+ blockId: BlockId,
values: Iterator[Any],
serializer: Serializer = defaultSerializer): ByteBuffer = {
val byteStream = new FastByteArrayOutputStream(4096)
@@ -951,7 +1003,7 @@ private[spark] class BlockManager(
* the iterator is reached.
*/
def dataDeserialize(
- blockId: String,
+ blockId: BlockId,
bytes: ByteBuffer,
serializer: Serializer = defaultSerializer): Iterator[Any] = {
bytes.rewind()
@@ -969,6 +1021,7 @@ private[spark] class BlockManager(
memoryStore.clear()
diskStore.clear()
metadataCleaner.cancel()
+ broadcastCleaner.cancel()
logInfo("BlockManager stopped")
}
}
@@ -1005,10 +1058,10 @@ private[spark] object BlockManager extends Logging {
}
def blockIdsToBlockManagers(
- blockIds: Array[String],
+ blockIds: Array[BlockId],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null)
- : Map[String, Seq[BlockManagerId]] =
+ : Map[BlockId, Seq[BlockManagerId]] =
{
// env == null and blockManagerMaster != null is used in tests
assert (env != null || blockManagerMaster != null)
@@ -1018,7 +1071,7 @@ private[spark] object BlockManager extends Logging {
blockManagerMaster.getLocations(blockIds)
}
- val blockManagers = new HashMap[String, Seq[BlockManagerId]]
+ val blockManagers = new HashMap[BlockId, Seq[BlockManagerId]]
for (i <- 0 until blockIds.length) {
blockManagers(blockIds(i)) = blockLocations(i)
}
@@ -1026,19 +1079,19 @@ private[spark] object BlockManager extends Logging {
}
def blockIdsToExecutorIds(
- blockIds: Array[String],
+ blockIds: Array[BlockId],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null)
- : Map[String, Seq[String]] =
+ : Map[BlockId, Seq[String]] =
{
blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId))
}
def blockIdsToHosts(
- blockIds: Array[String],
+ blockIds: Array[BlockId],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null)
- : Map[String, Seq[String]] =
+ : Map[BlockId, Seq[String]] =
{
blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host))
}
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 cf463d6ffc..94038649b3 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -60,7 +60,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
def updateBlockInfo(
blockManagerId: BlockManagerId,
- blockId: String,
+ blockId: BlockId,
storageLevel: StorageLevel,
memSize: Long,
diskSize: Long): Boolean = {
@@ -71,12 +71,12 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
}
/** Get locations of the blockId from the driver */
- def getLocations(blockId: String): Seq[BlockManagerId] = {
+ def getLocations(blockId: BlockId): Seq[BlockManagerId] = {
askDriverWithReply[Seq[BlockManagerId]](GetLocations(blockId))
}
/** Get locations of multiple blockIds from the driver */
- def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
+ def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds))
}
@@ -94,7 +94,7 @@ private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Loggi
* Remove a block from the slaves that have it. This can only be used to remove
* blocks that the driver knows about.
*/
- def removeBlock(blockId: String) {
+ def removeBlock(blockId: BlockId) {
askDriverWithReply(RemoveBlock(blockId))
}
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 c7b23ab094..633230c0a8 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -48,7 +48,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId]
// Mapping from block id to the set of block managers that have the block.
- private val blockLocations = new JHashMap[String, mutable.HashSet[BlockManagerId]]
+ private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]]
val akkaTimeout = Duration.create(
System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
@@ -129,10 +129,9 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
// First remove the metadata for the given RDD, and then asynchronously remove the blocks
// from the slaves.
- val prefix = "rdd_" + rddId + "_"
// Find all blocks for the given RDD, remove the block from both blockLocations and
// the blockManagerInfo that is tracking the blocks.
- val blocks = blockLocations.keySet().filter(_.startsWith(prefix))
+ val blocks = blockLocations.keys.flatMap(_.asRDDId).filter(_.rddId == rddId)
blocks.foreach { blockId =>
val bms: mutable.HashSet[BlockManagerId] = blockLocations.get(blockId)
bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId)))
@@ -198,7 +197,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
// Remove a block from the slaves that have it. This can only be used to remove
// blocks that the master knows about.
- private def removeBlockFromWorkers(blockId: String) {
+ private def removeBlockFromWorkers(blockId: BlockId) {
val locations = blockLocations.get(blockId)
if (locations != null) {
locations.foreach { blockManagerId: BlockManagerId =>
@@ -247,7 +246,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
private def updateBlockInfo(
blockManagerId: BlockManagerId,
- blockId: String,
+ blockId: BlockId,
storageLevel: StorageLevel,
memSize: Long,
diskSize: Long) {
@@ -292,11 +291,11 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
sender ! true
}
- private def getLocations(blockId: String): Seq[BlockManagerId] = {
+ private def getLocations(blockId: BlockId): Seq[BlockManagerId] = {
if (blockLocations.containsKey(blockId)) blockLocations.get(blockId).toSeq else Seq.empty
}
- private def getLocationsMultipleBlockIds(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
+ private def getLocationsMultipleBlockIds(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
blockIds.map(blockId => getLocations(blockId))
}
@@ -330,7 +329,7 @@ object BlockManagerMasterActor {
private var _remainingMem: Long = maxMem
// Mapping from block id to its status.
- private val _blocks = new JHashMap[String, BlockStatus]
+ private val _blocks = new JHashMap[BlockId, BlockStatus]
logInfo("Registering block manager %s with %s RAM".format(
blockManagerId.hostPort, Utils.bytesToString(maxMem)))
@@ -339,7 +338,7 @@ object BlockManagerMasterActor {
_lastSeenMs = System.currentTimeMillis()
}
- def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long,
+ def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long,
diskSize: Long) {
updateLastSeenMs()
@@ -383,7 +382,7 @@ object BlockManagerMasterActor {
}
}
- def removeBlock(blockId: String) {
+ def removeBlock(blockId: BlockId) {
if (_blocks.containsKey(blockId)) {
_remainingMem += _blocks.get(blockId).memSize
_blocks.remove(blockId)
@@ -394,7 +393,7 @@ object BlockManagerMasterActor {
def lastSeenMs: Long = _lastSeenMs
- def blocks: JHashMap[String, BlockStatus] = _blocks
+ def blocks: JHashMap[BlockId, BlockStatus] = _blocks
override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
index 24333a179c..45f51da288 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
@@ -30,7 +30,7 @@ private[storage] object BlockManagerMessages {
// Remove a block from the slaves that have it. This can only be used to remove
// blocks that the master knows about.
- case class RemoveBlock(blockId: String) extends ToBlockManagerSlave
+ case class RemoveBlock(blockId: BlockId) extends ToBlockManagerSlave
// Remove all blocks belonging to a specific RDD.
case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave
@@ -51,7 +51,7 @@ private[storage] object BlockManagerMessages {
class UpdateBlockInfo(
var blockManagerId: BlockManagerId,
- var blockId: String,
+ var blockId: BlockId,
var storageLevel: StorageLevel,
var memSize: Long,
var diskSize: Long)
@@ -62,7 +62,7 @@ private[storage] object BlockManagerMessages {
override def writeExternal(out: ObjectOutput) {
blockManagerId.writeExternal(out)
- out.writeUTF(blockId)
+ out.writeUTF(blockId.name)
storageLevel.writeExternal(out)
out.writeLong(memSize)
out.writeLong(diskSize)
@@ -70,7 +70,7 @@ private[storage] object BlockManagerMessages {
override def readExternal(in: ObjectInput) {
blockManagerId = BlockManagerId(in)
- blockId = in.readUTF()
+ blockId = BlockId(in.readUTF())
storageLevel = StorageLevel(in)
memSize = in.readLong()
diskSize = in.readLong()
@@ -79,7 +79,7 @@ private[storage] object BlockManagerMessages {
object UpdateBlockInfo {
def apply(blockManagerId: BlockManagerId,
- blockId: String,
+ blockId: BlockId,
storageLevel: StorageLevel,
memSize: Long,
diskSize: Long): UpdateBlockInfo = {
@@ -87,14 +87,14 @@ private[storage] object BlockManagerMessages {
}
// For pattern-matching
- def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = {
+ def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, BlockId, StorageLevel, Long, Long)] = {
Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize))
}
}
- case class GetLocations(blockId: String) extends ToBlockManagerMaster
+ case class GetLocations(blockId: BlockId) extends ToBlockManagerMaster
- case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster
+ case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster
case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
index 3d709cfde4..365866d1e3 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala
@@ -20,13 +20,15 @@ package org.apache.spark.storage
import com.codahale.metrics.{Gauge,MetricRegistry}
import org.apache.spark.metrics.source.Source
+import org.apache.spark.SparkContext
-private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source {
+private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext)
+ extends Source {
val metricRegistry = new MetricRegistry()
- val sourceName = "BlockManager"
+ val sourceName = "%s.BlockManager".format(sc.appName)
- metricRegistry.register(MetricRegistry.name("memory", "maxMem", "MBytes"), new Gauge[Long] {
+ metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _)
@@ -34,7 +36,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends
}
})
- metricRegistry.register(MetricRegistry.name("memory", "remainingMem", "MBytes"), new Gauge[Long] {
+ metricRegistry.register(MetricRegistry.name("memory", "remainingMem_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _)
@@ -42,7 +44,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends
}
})
- metricRegistry.register(MetricRegistry.name("memory", "memUsed", "MBytes"), new Gauge[Long] {
+ metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _)
@@ -51,7 +53,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager) extends
}
})
- metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed", "MBytes"), new Gauge[Long] {
+ metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] {
override def getValue: Long = {
val storageStatusList = blockManager.master.getStorageStatus
val diskSpaceUsed = storageStatusList
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
index 678c38203c..0c66addf9d 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
@@ -77,7 +77,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
}
}
- private def putBlock(id: String, bytes: ByteBuffer, level: StorageLevel) {
+ private def putBlock(id: BlockId, bytes: ByteBuffer, level: StorageLevel) {
val startTimeMs = System.currentTimeMillis()
logDebug("PutBlock " + id + " started from " + startTimeMs + " with data: " + bytes)
blockManager.putBytes(id, bytes, level)
@@ -85,7 +85,7 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
+ " with data size: " + bytes.limit)
}
- private def getBlock(id: String): ByteBuffer = {
+ private def getBlock(id: BlockId): ByteBuffer = {
val startTimeMs = System.currentTimeMillis()
logDebug("GetBlock " + id + " started from " + startTimeMs)
val buffer = blockManager.getLocalBytes(id) match {
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
index d8fa6a91d1..80dcb5a207 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala
@@ -24,9 +24,9 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.spark.network._
-private[spark] case class GetBlock(id: String)
-private[spark] case class GotBlock(id: String, data: ByteBuffer)
-private[spark] case class PutBlock(id: String, data: ByteBuffer, level: StorageLevel)
+private[spark] case class GetBlock(id: BlockId)
+private[spark] case class GotBlock(id: BlockId, data: ByteBuffer)
+private[spark] case class PutBlock(id: BlockId, data: ByteBuffer, level: StorageLevel)
private[spark] class BlockMessage() {
// Un-initialized: typ = 0
@@ -34,7 +34,7 @@ private[spark] class BlockMessage() {
// GotBlock: typ = 2
// PutBlock: typ = 3
private var typ: Int = BlockMessage.TYPE_NON_INITIALIZED
- private var id: String = null
+ private var id: BlockId = null
private var data: ByteBuffer = null
private var level: StorageLevel = null
@@ -74,7 +74,7 @@ private[spark] class BlockMessage() {
for (i <- 1 to idLength) {
idBuilder += buffer.getChar()
}
- id = idBuilder.toString()
+ id = BlockId(idBuilder.toString)
if (typ == BlockMessage.TYPE_PUT_BLOCK) {
@@ -109,28 +109,17 @@ private[spark] class BlockMessage() {
set(buffer)
}
- def getType: Int = {
- return typ
- }
-
- def getId: String = {
- return id
- }
-
- def getData: ByteBuffer = {
- return data
- }
-
- def getLevel: StorageLevel = {
- return level
- }
+ def getType: Int = typ
+ def getId: BlockId = id
+ def getData: ByteBuffer = data
+ def getLevel: StorageLevel = level
def toBufferMessage: BufferMessage = {
val startTime = System.currentTimeMillis
val buffers = new ArrayBuffer[ByteBuffer]()
- var buffer = ByteBuffer.allocate(4 + 4 + id.length() * 2)
- buffer.putInt(typ).putInt(id.length())
- id.foreach((x: Char) => buffer.putChar(x))
+ var buffer = ByteBuffer.allocate(4 + 4 + id.name.length * 2)
+ buffer.putInt(typ).putInt(id.name.length)
+ id.name.foreach((x: Char) => buffer.putChar(x))
buffer.flip()
buffers += buffer
@@ -212,7 +201,8 @@ private[spark] object BlockMessage {
def main(args: Array[String]) {
val B = new BlockMessage()
- B.set(new PutBlock("ABC", ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2))
+ val blockId = TestBlockId("ABC")
+ B.set(new PutBlock(blockId, ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2))
val bMsg = B.toBufferMessage
val C = new BlockMessage()
C.set(bMsg)
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
index 0aaf846b5b..6ce9127c74 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
@@ -111,14 +111,15 @@ private[spark] object BlockMessageArray {
}
def main(args: Array[String]) {
- val blockMessages =
+ val blockMessages =
(0 until 10).map { i =>
if (i % 2 == 0) {
val buffer = ByteBuffer.allocate(100)
buffer.clear
- BlockMessage.fromPutBlock(PutBlock(i.toString, buffer, StorageLevel.MEMORY_ONLY_SER))
+ BlockMessage.fromPutBlock(PutBlock(TestBlockId(i.toString), buffer,
+ StorageLevel.MEMORY_ONLY_SER))
} else {
- BlockMessage.fromGetBlock(GetBlock(i.toString))
+ BlockMessage.fromGetBlock(GetBlock(TestBlockId(i.toString)))
}
}
val blockMessageArray = new BlockMessageArray(blockMessages)
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 39f103297f..2a67800c45 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -25,7 +25,7 @@ package org.apache.spark.storage
*
* This interface does not support concurrent writes.
*/
-abstract class BlockObjectWriter(val blockId: String) {
+abstract class BlockObjectWriter(val blockId: BlockId) {
var closeEventHandler: () => Unit = _
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
index fa834371f4..ea42656240 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala
@@ -27,7 +27,7 @@ import org.apache.spark.Logging
*/
private[spark]
abstract class BlockStore(val blockManager: BlockManager) extends Logging {
- def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel)
+ def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel)
/**
* Put in a block and, possibly, also return its content as either bytes or another Iterator.
@@ -36,26 +36,26 @@ abstract class BlockStore(val blockManager: BlockManager) extends Logging {
* @return a PutResult that contains the size of the data, as well as the values put if
* returnValues is true (if not, the result's data field can be null)
*/
- def putValues(blockId: String, values: ArrayBuffer[Any], level: StorageLevel,
+ def putValues(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel,
returnValues: Boolean) : PutResult
/**
* Return the size of a block in bytes.
*/
- def getSize(blockId: String): Long
+ def getSize(blockId: BlockId): Long
- def getBytes(blockId: String): Option[ByteBuffer]
+ def getBytes(blockId: BlockId): Option[ByteBuffer]
- def getValues(blockId: String): Option[Iterator[Any]]
+ def getValues(blockId: BlockId): Option[Iterator[Any]]
/**
* Remove a block, if it exists.
* @param blockId the block to remove.
* @return True if the block was found and removed, False otherwise.
*/
- def remove(blockId: String): Boolean
+ def remove(blockId: BlockId): Boolean
- def contains(blockId: String): Boolean
+ def contains(blockId: BlockId): Boolean
def clear() { }
}
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
index fc25ef0fae..b7ca61e938 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -42,7 +42,7 @@ import org.apache.spark.util.Utils
private class DiskStore(blockManager: BlockManager, rootDirs: String)
extends BlockStore(blockManager) with Logging {
- class DiskBlockObjectWriter(blockId: String, serializer: Serializer, bufferSize: Int)
+ class DiskBlockObjectWriter(blockId: BlockId, serializer: Serializer, bufferSize: Int)
extends BlockObjectWriter(blockId) {
private val f: File = createFile(blockId /*, allowAppendExisting */)
@@ -124,16 +124,16 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
addShutdownHook()
- def getBlockWriter(blockId: String, serializer: Serializer, bufferSize: Int)
+ def getBlockWriter(blockId: BlockId, serializer: Serializer, bufferSize: Int)
: BlockObjectWriter = {
new DiskBlockObjectWriter(blockId, serializer, bufferSize)
}
- override def getSize(blockId: String): Long = {
+ override def getSize(blockId: BlockId): Long = {
getFile(blockId).length()
}
- override def putBytes(blockId: String, _bytes: ByteBuffer, level: StorageLevel) {
+ override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) {
// So that we do not modify the input offsets !
// duplicate does not copy buffer, so inexpensive
val bytes = _bytes.duplicate()
@@ -163,7 +163,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
}
override def putValues(
- blockId: String,
+ blockId: BlockId,
values: ArrayBuffer[Any],
level: StorageLevel,
returnValues: Boolean)
@@ -192,13 +192,13 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
}
}
- override def getBytes(blockId: String): Option[ByteBuffer] = {
+ override def getBytes(blockId: BlockId): Option[ByteBuffer] = {
val file = getFile(blockId)
val bytes = getFileBytes(file)
Some(bytes)
}
- override def getValues(blockId: String): Option[Iterator[Any]] = {
+ override def getValues(blockId: BlockId): Option[Iterator[Any]] = {
getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes))
}
@@ -206,11 +206,11 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
* A version of getValues that allows a custom serializer. This is used as part of the
* shuffle short-circuit code.
*/
- def getValues(blockId: String, serializer: Serializer): Option[Iterator[Any]] = {
+ def getValues(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = {
getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes, serializer))
}
- override def remove(blockId: String): Boolean = {
+ override def remove(blockId: BlockId): Boolean = {
val file = getFile(blockId)
if (file.exists()) {
file.delete()
@@ -219,11 +219,11 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
}
}
- override def contains(blockId: String): Boolean = {
+ override def contains(blockId: BlockId): Boolean = {
getFile(blockId).exists()
}
- private def createFile(blockId: String, allowAppendExisting: Boolean = false): File = {
+ private def createFile(blockId: BlockId, allowAppendExisting: Boolean = false): File = {
val file = getFile(blockId)
if (!allowAppendExisting && file.exists()) {
// NOTE(shivaram): Delete the file if it exists. This might happen if a ShuffleMap task
@@ -234,11 +234,11 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
file
}
- private def getFile(blockId: String): File = {
+ private def getFile(blockId: BlockId): File = {
logDebug("Getting file for block " + blockId)
// Figure out which local directory it hashes to, and which subdirectory in that
- val hash = math.abs(blockId.hashCode)
+ val hash = Utils.nonNegativeHash(blockId)
val dirId = hash % localDirs.length
val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
@@ -258,7 +258,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
}
}
- new File(subDir, blockId)
+ new File(subDir, blockId.name)
}
private def createLocalDirs(): Array[File] = {
@@ -307,7 +307,7 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
}
}
if (shuffleSender != null) {
- shuffleSender.stop
+ shuffleSender.stop()
}
}
})
@@ -315,11 +315,10 @@ private class DiskStore(blockManager: BlockManager, rootDirs: String)
private[storage] def startShuffleBlockSender(port: Int): Int = {
val pResolver = new PathResolver {
- override def getAbsolutePath(blockId: String): String = {
- if (!blockId.startsWith("shuffle_")) {
- return null
- }
- DiskStore.this.getFile(blockId).getAbsolutePath()
+ override def getAbsolutePath(blockIdString: String): String = {
+ val blockId = BlockId(blockIdString)
+ if (!blockId.isShuffle) null
+ else DiskStore.this.getFile(blockId).getAbsolutePath
}
}
shuffleSender = new ShuffleSender(port, pResolver)
diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
index 3b3b2342fa..05f676c6e2 100644
--- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
@@ -30,10 +30,10 @@ import org.apache.spark.util.{SizeEstimator, Utils}
private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
extends BlockStore(blockManager) {
- case class Entry(value: Any, size: Long, deserialized: Boolean, var dropPending: Boolean = false)
+ case class Entry(value: Any, size: Long, deserialized: Boolean)
- private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true)
- private var currentMemory = 0L
+ private val entries = new LinkedHashMap[BlockId, Entry](32, 0.75f, true)
+ @volatile private var currentMemory = 0L
// Object used to ensure that only one thread is putting blocks and if necessary, dropping
// blocks from the memory store.
private val putLock = new Object()
@@ -42,13 +42,13 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
def freeMemory: Long = maxMemory - currentMemory
- override def getSize(blockId: String): Long = {
+ override def getSize(blockId: BlockId): Long = {
entries.synchronized {
entries.get(blockId).size
}
}
- override def putBytes(blockId: String, _bytes: ByteBuffer, level: StorageLevel) {
+ override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) {
// Work on a duplicate - since the original input might be used elsewhere.
val bytes = _bytes.duplicate()
bytes.rewind()
@@ -64,7 +64,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
}
override def putValues(
- blockId: String,
+ blockId: BlockId,
values: ArrayBuffer[Any],
level: StorageLevel,
returnValues: Boolean)
@@ -81,7 +81,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
}
}
- override def getBytes(blockId: String): Option[ByteBuffer] = {
+ override def getBytes(blockId: BlockId): Option[ByteBuffer] = {
val entry = entries.synchronized {
entries.get(blockId)
}
@@ -94,7 +94,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
}
}
- override def getValues(blockId: String): Option[Iterator[Any]] = {
+ override def getValues(blockId: BlockId): Option[Iterator[Any]] = {
val entry = entries.synchronized {
entries.get(blockId)
}
@@ -108,11 +108,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
}
}
- override def remove(blockId: String): Boolean = {
+ override def remove(blockId: BlockId): Boolean = {
entries.synchronized {
- val entry = entries.get(blockId)
+ val entry = entries.remove(blockId)
if (entry != null) {
- entries.remove(blockId)
currentMemory -= entry.size
logInfo("Block %s of size %d dropped from memory (free %d)".format(
blockId, entry.size, freeMemory))
@@ -126,19 +125,16 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
override def clear() {
entries.synchronized {
entries.clear()
+ currentMemory = 0
}
logInfo("MemoryStore cleared")
}
/**
- * Return the RDD ID that a given block ID is from, or null if it is not an RDD block.
+ * Return the RDD ID that a given block ID is from, or None if it is not an RDD block.
*/
- private def getRddId(blockId: String): String = {
- if (blockId.startsWith("rdd_")) {
- blockId.split('_')(1)
- } else {
- null
- }
+ private def getRddId(blockId: BlockId): Option[Int] = {
+ blockId.asRDDId.map(_.rddId)
}
/**
@@ -151,7 +147,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
* blocks to free memory for one block, another thread may use up the freed space for
* another block.
*/
- private def tryToPut(blockId: String, value: Any, size: Long, deserialized: Boolean): Boolean = {
+ private def tryToPut(blockId: BlockId, value: Any, size: Long, deserialized: Boolean): Boolean = {
// TODO: Its possible to optimize the locking by locking entries only when selecting blocks
// to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has been
// released, it must be ensured that those to-be-dropped blocks are not double counted for
@@ -160,8 +156,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
putLock.synchronized {
if (ensureFreeSpace(blockId, size)) {
val entry = new Entry(value, size, deserialized)
- entries.synchronized { entries.put(blockId, entry) }
- currentMemory += size
+ entries.synchronized {
+ entries.put(blockId, entry)
+ currentMemory += size
+ }
if (deserialized) {
logInfo("Block %s stored as values to memory (estimated size %s, free %s)".format(
blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory)))
@@ -193,7 +191,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
* Assumes that a lock is held by the caller to ensure only one thread is dropping blocks.
* Otherwise, the freed space may fill up before the caller puts in their new value.
*/
- private def ensureFreeSpace(blockIdToAdd: String, space: Long): Boolean = {
+ private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): Boolean = {
logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format(
space, currentMemory, maxMemory))
@@ -205,7 +203,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
if (maxMemory - currentMemory < space) {
val rddToAdd = getRddId(blockIdToAdd)
- val selectedBlocks = new ArrayBuffer[String]()
+ val selectedBlocks = new ArrayBuffer[BlockId]()
var selectedMemory = 0L
// This is synchronized to ensure that the set of entries is not changed
@@ -216,7 +214,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) {
val pair = iterator.next()
val blockId = pair.getKey
- if (rddToAdd != null && rddToAdd == getRddId(blockId)) {
+ if (rddToAdd != None && rddToAdd == getRddId(blockId)) {
logInfo("Will not store " + blockIdToAdd + " as it would require dropping another " +
"block from the same RDD")
return false
@@ -250,7 +248,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
return true
}
- override def contains(blockId: String): Boolean = {
+ override def contains(blockId: BlockId): Boolean = {
entries.synchronized { entries.containsKey(blockId) }
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
index 9da11efb57..f39fcd87fb 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -30,7 +30,6 @@ trait ShuffleBlocks {
def releaseWriters(group: ShuffleWriterGroup)
}
-
private[spark]
class ShuffleBlockManager(blockManager: BlockManager) {
@@ -40,7 +39,7 @@ class ShuffleBlockManager(blockManager: BlockManager) {
override def acquireWriters(mapId: Int): ShuffleWriterGroup = {
val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024
val writers = Array.tabulate[BlockObjectWriter](numBuckets) { bucketId =>
- val blockId = ShuffleBlockManager.blockId(shuffleId, bucketId, mapId)
+ val blockId = ShuffleBlockId(shuffleId, mapId, bucketId)
blockManager.getDiskBlockWriter(blockId, serializer, bufferSize)
}
new ShuffleWriterGroup(mapId, writers)
@@ -52,16 +51,3 @@ class ShuffleBlockManager(blockManager: BlockManager) {
}
}
}
-
-
-private[spark]
-object ShuffleBlockManager {
-
- // Returns the block id for a given shuffle block.
- def blockId(shuffleId: Int, bucketId: Int, groupId: Int): String = {
- "shuffle_" + shuffleId + "_" + groupId + "_" + bucketId
- }
-
- // Returns true if the block is a shuffle block.
- def isShuffle(blockId: String): Boolean = blockId.startsWith("shuffle_")
-}
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
index 755f1a760e..632ff047d1 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
@@ -23,9 +23,9 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput}
* Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory,
* whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory
* in a serialized format, and whether to replicate the RDD partitions on multiple nodes.
- * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants for
- * commonly useful storage levels. To create your own storage level object, use the factor method
- * of the singleton object (`StorageLevel(...)`).
+ * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants
+ * for commonly useful storage levels. To create your own storage level object, use the
+ * factory method of the singleton object (`StorageLevel(...)`).
*/
class StorageLevel private(
private var useDisk_ : Boolean,
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
index 2bb7715696..1720007e4e 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
@@ -23,20 +23,24 @@ import org.apache.spark.util.Utils
private[spark]
case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long,
- blocks: Map[String, BlockStatus]) {
+ blocks: Map[BlockId, BlockStatus]) {
- def memUsed(blockPrefix: String = "") = {
- blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.memSize).
- reduceOption(_+_).getOrElse(0l)
- }
+ def memUsed() = blocks.values.map(_.memSize).reduceOption(_+_).getOrElse(0L)
- def diskUsed(blockPrefix: String = "") = {
- blocks.filterKeys(_.startsWith(blockPrefix)).values.map(_.diskSize).
- reduceOption(_+_).getOrElse(0l)
- }
+ def memUsedByRDD(rddId: Int) =
+ rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_+_).getOrElse(0L)
+
+ def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_+_).getOrElse(0L)
+
+ def diskUsedByRDD(rddId: Int) =
+ rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_+_).getOrElse(0L)
def memRemaining : Long = maxMem - memUsed()
+ def rddBlocks = blocks.flatMap {
+ case (rdd: RDDBlockId, status) => Some(rdd, status)
+ case _ => None
+ }
}
case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
@@ -60,7 +64,7 @@ object StorageUtils {
/* Returns RDD-level information, compiled from a list of StorageStatus objects */
def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus],
sc: SparkContext) : Array[RDDInfo] = {
- rddInfoFromBlockStatusList(storageStatusList.flatMap(_.blocks).toMap, sc)
+ rddInfoFromBlockStatusList(storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc)
}
/* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */
@@ -71,26 +75,21 @@ object StorageUtils {
}
/* Given a list of BlockStatus objets, returns information for each RDD */
- def rddInfoFromBlockStatusList(infos: Map[String, BlockStatus],
+ def rddInfoFromBlockStatusList(infos: Map[RDDBlockId, BlockStatus],
sc: SparkContext) : Array[RDDInfo] = {
// Group by rddId, ignore the partition name
- val groupedRddBlocks = infos.filterKeys(_.startsWith("rdd_")).groupBy { case(k, v) =>
- k.substring(0,k.lastIndexOf('_'))
- }.mapValues(_.values.toArray)
+ val groupedRddBlocks = infos.groupBy { case(k, v) => k.rddId }.mapValues(_.values.toArray)
// For each RDD, generate an RDDInfo object
- val rddInfos = groupedRddBlocks.map { case (rddKey, rddBlocks) =>
+ val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) =>
// Add up memory and disk sizes
val memSize = rddBlocks.map(_.memSize).reduce(_ + _)
val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _)
- // Find the id of the RDD, e.g. rdd_1 => 1
- val rddId = rddKey.split("_").last.toInt
-
// Get the friendly name and storage level for the RDD, if available
sc.persistentRdds.get(rddId).map { r =>
- val rddName = Option(r.name).getOrElse(rddKey)
+ val rddName = Option(r.name).getOrElse(rddId.toString)
val rddStorageLevel = r.getStorageLevel
RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, memSize, diskSize)
}
@@ -101,16 +100,14 @@ object StorageUtils {
rddInfos
}
- /* Removes all BlockStatus object that are not part of a block prefix */
- def filterStorageStatusByPrefix(storageStatusList: Array[StorageStatus],
- prefix: String) : Array[StorageStatus] = {
+ /* Filters storage status by a given RDD id. */
+ def filterStorageStatusByRDD(storageStatusList: Array[StorageStatus], rddId: Int)
+ : Array[StorageStatus] = {
storageStatusList.map { status =>
- val newBlocks = status.blocks.filterKeys(_.startsWith(prefix))
+ val newBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toMap[BlockId, BlockStatus]
//val newRemainingMem = status.maxMem - newBlocks.values.map(_.memSize).reduce(_ + _)
StorageStatus(status.blockManagerId, status.maxMem, newBlocks)
}
-
}
-
}
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..860e680576 100644
--- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
@@ -36,11 +36,11 @@ private[spark] object ThreadingTest {
val numBlocksPerProducer = 20000
private[spark] class ProducerThread(manager: BlockManager, id: Int) extends Thread {
- val queue = new ArrayBlockingQueue[(String, Seq[Int])](100)
+ val queue = new ArrayBlockingQueue[(BlockId, Seq[Int])](100)
override def run() {
for (i <- 1 to numBlocksPerProducer) {
- val blockId = "b-" + id + "-" + i
+ val blockId = TestBlockId("b-" + id + "-" + i)
val blockSize = Random.nextInt(1000)
val block = (1 to blockSize).map(_ => Random.nextInt())
val level = randomLevel()
@@ -64,7 +64,7 @@ private[spark] object ThreadingTest {
private[spark] class ConsumerThread(
manager: BlockManager,
- queue: ArrayBlockingQueue[(String, Seq[Int])]
+ queue: ArrayBlockingQueue[(BlockId, Seq[Int])]
) extends Thread {
var numBlockConsumed = 0
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index ad456ea565..f1d86c0221 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -1,4 +1,4 @@
-/*
+/*
* 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.
@@ -79,10 +79,11 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging {
server.foreach(_.stop())
}
- private[spark] def appUIAddress = "http://" + host + ":" + boundPort.getOrElse("-1")
+ private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1")
+
}
private[spark] object SparkUI {
- val DEFAULT_PORT = "3030"
+ val DEFAULT_PORT = "4040"
val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static"
}
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index ce1acf564c..5573b3847b 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -25,38 +25,45 @@ import org.apache.spark.SparkContext
private[spark] object UIUtils {
import Page._
+ // Yarn has to go through a proxy so the base uri is provided and has to be on all links
+ private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).
+ getOrElse("")
+
+ def prependBaseUri(resource: String = "") = uiRoot + resource
+
/** Returns a spark page with correctly formatted headers */
def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value)
: Seq[Node] = {
val jobs = page match {
- case Stages => <li class="active"><a href="/stages">Stages</a></li>
- case _ => <li><a href="/stages">Stages</a></li>
+ case Stages => <li class="active"><a href={prependBaseUri("/stages")}>Stages</a></li>
+ case _ => <li><a href={prependBaseUri("/stages")}>Stages</a></li>
}
val storage = page match {
- case Storage => <li class="active"><a href="/storage">Storage</a></li>
- case _ => <li><a href="/storage">Storage</a></li>
+ case Storage => <li class="active"><a href={prependBaseUri("/storage")}>Storage</a></li>
+ case _ => <li><a href={prependBaseUri("/storage")}>Storage</a></li>
}
val environment = page match {
- case Environment => <li class="active"><a href="/environment">Environment</a></li>
- case _ => <li><a href="/environment">Environment</a></li>
+ case Environment =>
+ <li class="active"><a href={prependBaseUri("/environment")}>Environment</a></li>
+ case _ => <li><a href={prependBaseUri("/environment")}>Environment</a></li>
}
val executors = page match {
- case Executors => <li class="active"><a href="/executors">Executors</a></li>
- case _ => <li><a href="/executors">Executors</a></li>
+ case Executors => <li class="active"><a href={prependBaseUri("/executors")}>Executors</a></li>
+ case _ => <li><a href={prependBaseUri("/executors")}>Executors</a></li>
}
<html>
<head>
<meta http-equiv="Content-type" content="text/html; charset=utf-8" />
- <link rel="stylesheet" href="/static/bootstrap.min.css" type="text/css" />
- <link rel="stylesheet" href="/static/webui.css" type="text/css" />
- <script src="/static/sorttable.js"></script>
+ <link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")} type="text/css" />
+ <link rel="stylesheet" href={prependBaseUri("/static/webui.css")} type="text/css" />
+ <script src={prependBaseUri("/static/sorttable.js")} ></script>
<title>{sc.appName} - {title}</title>
</head>
<body>
<div class="navbar navbar-static-top">
<div class="navbar-inner">
- <a href="/" class="brand"><img src="/static/spark-logo-77x50px-hd.png" /></a>
+ <a href={prependBaseUri("/")} class="brand"><img src={prependBaseUri("/static/spark-logo-77x50px-hd.png")} /></a>
<ul class="nav">
{jobs}
{storage}
@@ -86,9 +93,9 @@ private[spark] object UIUtils {
<html>
<head>
<meta http-equiv="Content-type" content="text/html; charset=utf-8" />
- <link rel="stylesheet" href="/static/bootstrap.min.css" type="text/css" />
- <link rel="stylesheet" href="/static/webui.css" type="text/css" />
- <script src="/static/sorttable.js"></script>
+ <link rel="stylesheet" href={prependBaseUri("/static/bootstrap.min.css")} type="text/css" />
+ <link rel="stylesheet" href={prependBaseUri("/static/webui.css")} type="text/css" />
+ <script src={prependBaseUri("/static/sorttable.js")} ></script>
<title>{title}</title>
</head>
<body>
@@ -96,7 +103,7 @@ private[spark] object UIUtils {
<div class="row-fluid">
<div class="span12">
<h3 style="vertical-align: middle; display: inline-block;">
- <img src="/static/spark-logo-77x50px-hd.png" style="margin-right: 15px;" />
+ <img src={prependBaseUri("/static/spark-logo-77x50px-hd.png")} style="margin-right: 15px;" />
{title}
</h3>
</div>
diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
index 0ecb22d2f9..fcd1b518d0 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -21,7 +21,7 @@ import scala.util.Random
import org.apache.spark.SparkContext
import org.apache.spark.SparkContext._
-import org.apache.spark.scheduler.cluster.SchedulingMode
+import org.apache.spark.scheduler.SchedulingMode
/**
@@ -35,7 +35,7 @@ private[spark] object UIWorkloadGenerator {
def main(args: Array[String]) {
if (args.length < 2) {
- println("usage: ./spark-class spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+ println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
System.exit(1)
}
val master = args(0)
@@ -43,13 +43,13 @@ private[spark] object UIWorkloadGenerator {
val appName = "Spark UI Tester"
if (schedulingMode == SchedulingMode.FAIR) {
- System.setProperty("spark.cluster.schedulingmode", "FAIR")
+ System.setProperty("spark.scheduler.mode", "FAIR")
}
val sc = new SparkContext(master, appName)
def setProperties(s: String) = {
if(schedulingMode == SchedulingMode.FAIR) {
- sc.setLocalProperty("spark.scheduler.cluster.fair.pool", s)
+ sc.setLocalProperty("spark.scheduler.pool", s)
}
sc.setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, s)
}
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
index d1868dcf78..42e9be6e19 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
@@ -26,8 +26,8 @@ import org.eclipse.jetty.server.Handler
import org.apache.spark.{ExceptionFailure, Logging, SparkContext}
import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.scheduler.cluster.TaskInfo
import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListenerTaskEnd, SparkListener}
+import org.apache.spark.scheduler.TaskInfo
import org.apache.spark.ui.JettyUtils._
import org.apache.spark.ui.Page.Executors
import org.apache.spark.ui.UIUtils
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
index 3b428effaf..b39c0e9769 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
@@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.{NodeSeq, Node}
-import org.apache.spark.scheduler.cluster.SchedulingMode
+import org.apache.spark.scheduler.SchedulingMode
import org.apache.spark.ui.Page._
import org.apache.spark.ui.UIUtils._
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
index e2bcd98545..eb3b4e8522 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala
@@ -21,10 +21,8 @@ import scala.Seq
import scala.collection.mutable.{ListBuffer, HashMap, HashSet}
import org.apache.spark.{ExceptionFailure, SparkContext, Success}
-import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster.TaskInfo
import org.apache.spark.executor.TaskMetrics
-import collection.mutable
+import org.apache.spark.scheduler._
/**
* Tracks task-level information to be displayed in the UI.
@@ -95,7 +93,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
activeStages += stage
val poolName = Option(stageSubmitted.properties).map {
- p => p.getProperty("spark.scheduler.cluster.fair.pool", DEFAULT_POOL_NAME)
+ p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME)
}.getOrElse(DEFAULT_POOL_NAME)
stageToPool(stage) = poolName
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
index 6aecef5120..e7eab374ad 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
@@ -32,8 +32,8 @@ import org.apache.spark.ui.JettyUtils._
import org.apache.spark.{ExceptionFailure, SparkContext, Success}
import org.apache.spark.scheduler._
import collection.mutable
-import org.apache.spark.scheduler.cluster.SchedulingMode
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.scheduler.SchedulingMode
+import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.util.Utils
/** Web UI showing progress status of all jobs in the given SparkContext. */
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
index 5670c933bd..06810d8dbc 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala
@@ -21,8 +21,8 @@ import scala.collection.mutable.HashMap
import scala.collection.mutable.HashSet
import scala.xml.Node
-import org.apache.spark.scheduler.Stage
-import org.apache.spark.scheduler.cluster.Schedulable
+import org.apache.spark.scheduler.{Schedulable, Stage}
+import org.apache.spark.ui.UIUtils
/** Table showing list of pools */
private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) {
@@ -60,7 +60,7 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis
case None => 0
}
<tr>
- <td><a href={"/stages/pool?poolname=%s".format(p.name)}>{p.name}</a></td>
+ <td><a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(),p.name)}>{p.name}</a></td>
<td>{p.minShare}</td>
<td>{p.weight}</td>
<td>{activeStages}</td>
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 a9969ab1c0..163a3746ea 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
@@ -23,12 +23,12 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
+import org.apache.spark.{ExceptionFailure}
+import org.apache.spark.executor.TaskMetrics
import org.apache.spark.ui.UIUtils._
import org.apache.spark.ui.Page._
import org.apache.spark.util.{Utils, Distribution}
-import org.apache.spark.{ExceptionFailure}
-import org.apache.spark.scheduler.cluster.TaskInfo
-import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler.TaskInfo
/** Page showing statistics and task list for a given stage */
private[spark] class StagePage(parent: JobProgressUI) {
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
index a8911e46ae..07db8622da 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
@@ -22,8 +22,8 @@ import java.util.Date
import scala.xml.Node
import scala.collection.mutable.HashSet
-import org.apache.spark.scheduler.cluster.{SchedulingMode, TaskInfo}
-import org.apache.spark.scheduler.Stage
+import org.apache.spark.scheduler.{SchedulingMode, Stage, TaskInfo}
+import org.apache.spark.ui.UIUtils
import org.apache.spark.util.Utils
@@ -98,7 +98,8 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
val poolName = listener.stageToPool.get(s)
- val nameLink = <a href={"/stages/stage?id=%s".format(s.id)}>{s.name}</a>
+ val nameLink =
+ <a href={"%s/stages/stage?id=%s".format(UIUtils.prependBaseUri(),s.id)}>{s.name}</a>
val description = listener.stageToDescription.get(s)
.map(d => <div><em>{d}</em></div><div>{nameLink}</div>).getOrElse(nameLink)
val finishTime = s.completionTime.getOrElse(System.currentTimeMillis())
@@ -107,7 +108,8 @@ private[spark] class StageTable(val stages: Seq[Stage], val parent: JobProgressU
<tr>
<td>{s.id}</td>
{if (isFairScheduler) {
- <td><a href={"/stages/pool?poolname=%s".format(poolName.get)}>{poolName.get}</a></td>}
+ <td><a href={"%s/stages/pool?poolname=%s".format(UIUtils.prependBaseUri(),poolName.get)}>
+ {poolName.get}</a></td>}
}
<td>{description}</td>
<td valign="middle">{submissionTime}</td>
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
index c3ec907370..109a7d4094 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
@@ -50,7 +50,7 @@ private[spark] class IndexPage(parent: BlockManagerUI) {
def rddRow(rdd: RDDInfo): Seq[Node] = {
<tr>
<td>
- <a href={"/storage/rdd?id=%s".format(rdd.id)}>
+ <a href={"%s/storage/rdd?id=%s".format(prependBaseUri(),rdd.id)}>
{rdd.name}
</a>
</td>
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
index 43c1257677..b83cd54f3c 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala
@@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import org.apache.spark.storage.{StorageStatus, StorageUtils}
+import org.apache.spark.storage.{BlockId, StorageStatus, StorageUtils}
import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus
import org.apache.spark.ui.UIUtils._
import org.apache.spark.ui.Page._
@@ -33,21 +33,20 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
val sc = parent.sc
def render(request: HttpServletRequest): Seq[Node] = {
- val id = request.getParameter("id")
- val prefix = "rdd_" + id.toString
+ val id = request.getParameter("id").toInt
val storageStatusList = sc.getExecutorStorageStatus
- val filteredStorageStatusList = StorageUtils.
- filterStorageStatusByPrefix(storageStatusList, prefix)
+ val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id)
val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head
val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage")
- val workers = filteredStorageStatusList.map((prefix, _))
+ val workers = filteredStorageStatusList.map((id, _))
val workerTable = listingTable(workerHeaders, workerRow, workers)
val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk",
"Executors")
- val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.sortWith(_._1 < _._1)
+ val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray.
+ sortWith(_._1.name < _._1.name)
val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList)
val blocks = blockStatuses.map {
case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN")))
@@ -99,7 +98,7 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
headerSparkPage(content, parent.sc, "RDD Storage Info for " + rddInfo.name, Storage)
}
- def blockRow(row: (String, BlockStatus, Seq[String])): Seq[Node] = {
+ def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = {
val (id, block, locations) = row
<tr>
<td>{id}</td>
@@ -118,15 +117,15 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
</tr>
}
- def workerRow(worker: (String, StorageStatus)): Seq[Node] = {
- val (prefix, status) = worker
+ def workerRow(worker: (Int, StorageStatus)): Seq[Node] = {
+ val (rddId, status) = worker
<tr>
<td>{status.blockManagerId.host + ":" + status.blockManagerId.port}</td>
<td>
- {Utils.bytesToString(status.memUsed(prefix))}
+ {Utils.bytesToString(status.memUsedByRDD(rddId))}
({Utils.bytesToString(status.memRemaining)} Remaining)
</td>
- <td>{Utils.bytesToString(status.diskUsed(prefix))}</td>
+ <td>{Utils.bytesToString(status.diskUsedByRDD(rddId))}</td>
</tr>
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala
new file mode 100644
index 0000000000..f60deafc6f
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/AppendOnlyMap.scala
@@ -0,0 +1,230 @@
+/*
+ * 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
+
+/**
+ * A simple open hash table optimized for the append-only use case, where keys
+ * are never removed, but the value for each key may be changed.
+ *
+ * This implementation uses quadratic probing with a power-of-2 hash table
+ * size, which is guaranteed to explore all spaces for each key (see
+ * http://en.wikipedia.org/wiki/Quadratic_probing).
+ *
+ * TODO: Cache the hash values of each key? java.util.HashMap does that.
+ */
+private[spark]
+class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, V)] with Serializable {
+ require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements")
+ require(initialCapacity >= 1, "Invalid initial capacity")
+
+ private var capacity = nextPowerOf2(initialCapacity)
+ private var mask = capacity - 1
+ private var curSize = 0
+
+ // Holds keys and values in the same array for memory locality; specifically, the order of
+ // elements is key0, value0, key1, value1, key2, value2, etc.
+ private var data = new Array[AnyRef](2 * capacity)
+
+ // Treat the null key differently so we can use nulls in "data" to represent empty items.
+ private var haveNullValue = false
+ private var nullValue: V = null.asInstanceOf[V]
+
+ private val LOAD_FACTOR = 0.7
+
+ /** Get the value for a given key */
+ def apply(key: K): V = {
+ val k = key.asInstanceOf[AnyRef]
+ if (k.eq(null)) {
+ return nullValue
+ }
+ var pos = rehash(k.hashCode) & mask
+ var i = 1
+ while (true) {
+ val curKey = data(2 * pos)
+ if (k.eq(curKey) || k == curKey) {
+ return data(2 * pos + 1).asInstanceOf[V]
+ } else if (curKey.eq(null)) {
+ return null.asInstanceOf[V]
+ } else {
+ val delta = i
+ pos = (pos + delta) & mask
+ i += 1
+ }
+ }
+ return null.asInstanceOf[V]
+ }
+
+ /** Set the value for a key */
+ def update(key: K, value: V): Unit = {
+ val k = key.asInstanceOf[AnyRef]
+ if (k.eq(null)) {
+ if (!haveNullValue) {
+ incrementSize()
+ }
+ nullValue = value
+ haveNullValue = true
+ return
+ }
+ val isNewEntry = putInto(data, k, value.asInstanceOf[AnyRef])
+ if (isNewEntry) {
+ incrementSize()
+ }
+ }
+
+ /**
+ * Set the value for key to updateFunc(hadValue, oldValue), where oldValue will be the old value
+ * for key, if any, or null otherwise. Returns the newly updated value.
+ */
+ def changeValue(key: K, updateFunc: (Boolean, V) => V): V = {
+ val k = key.asInstanceOf[AnyRef]
+ if (k.eq(null)) {
+ if (!haveNullValue) {
+ incrementSize()
+ }
+ nullValue = updateFunc(haveNullValue, nullValue)
+ haveNullValue = true
+ return nullValue
+ }
+ var pos = rehash(k.hashCode) & mask
+ var i = 1
+ while (true) {
+ val curKey = data(2 * pos)
+ if (k.eq(curKey) || k == curKey) {
+ val newValue = updateFunc(true, data(2 * pos + 1).asInstanceOf[V])
+ data(2 * pos + 1) = newValue.asInstanceOf[AnyRef]
+ return newValue
+ } else if (curKey.eq(null)) {
+ val newValue = updateFunc(false, null.asInstanceOf[V])
+ data(2 * pos) = k
+ data(2 * pos + 1) = newValue.asInstanceOf[AnyRef]
+ incrementSize()
+ return newValue
+ } else {
+ val delta = i
+ pos = (pos + delta) & mask
+ i += 1
+ }
+ }
+ null.asInstanceOf[V] // Never reached but needed to keep compiler happy
+ }
+
+ /** Iterator method from Iterable */
+ override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] {
+ var pos = -1
+
+ /** Get the next value we should return from next(), or null if we're finished iterating */
+ def nextValue(): (K, V) = {
+ if (pos == -1) { // Treat position -1 as looking at the null value
+ if (haveNullValue) {
+ return (null.asInstanceOf[K], nullValue)
+ }
+ pos += 1
+ }
+ while (pos < capacity) {
+ if (!data(2 * pos).eq(null)) {
+ return (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V])
+ }
+ pos += 1
+ }
+ null
+ }
+
+ override def hasNext: Boolean = nextValue() != null
+
+ override def next(): (K, V) = {
+ val value = nextValue()
+ if (value == null) {
+ throw new NoSuchElementException("End of iterator")
+ }
+ pos += 1
+ value
+ }
+ }
+
+ override def size: Int = curSize
+
+ /** Increase table size by 1, rehashing if necessary */
+ private def incrementSize() {
+ curSize += 1
+ if (curSize > LOAD_FACTOR * capacity) {
+ growTable()
+ }
+ }
+
+ /**
+ * Re-hash a value to deal better with hash functions that don't differ
+ * in the lower bits, similar to java.util.HashMap
+ */
+ private def rehash(h: Int): Int = {
+ val r = h ^ (h >>> 20) ^ (h >>> 12)
+ r ^ (r >>> 7) ^ (r >>> 4)
+ }
+
+ /**
+ * Put an entry into a table represented by data, returning true if
+ * this increases the size of the table or false otherwise. Assumes
+ * that "data" has at least one empty slot.
+ */
+ private def putInto(data: Array[AnyRef], key: AnyRef, value: AnyRef): Boolean = {
+ val mask = (data.length / 2) - 1
+ var pos = rehash(key.hashCode) & mask
+ var i = 1
+ while (true) {
+ val curKey = data(2 * pos)
+ if (curKey.eq(null)) {
+ data(2 * pos) = key
+ data(2 * pos + 1) = value.asInstanceOf[AnyRef]
+ return true
+ } else if (curKey.eq(key) || curKey == key) {
+ data(2 * pos + 1) = value.asInstanceOf[AnyRef]
+ return false
+ } else {
+ val delta = i
+ pos = (pos + delta) & mask
+ i += 1
+ }
+ }
+ return false // Never reached but needed to keep compiler happy
+ }
+
+ /** Double the table's size and re-hash everything */
+ private def growTable() {
+ val newCapacity = capacity * 2
+ if (newCapacity >= (1 << 30)) {
+ // We can't make the table this big because we want an array of 2x
+ // that size for our data, but array sizes are at most Int.MaxValue
+ throw new Exception("Can't make capacity bigger than 2^29 elements")
+ }
+ val newData = new Array[AnyRef](2 * newCapacity)
+ var pos = 0
+ while (pos < capacity) {
+ if (!data(2 * pos).eq(null)) {
+ putInto(newData, data(2 * pos), data(2 * pos + 1))
+ }
+ pos += 1
+ }
+ data = newData
+ capacity = newCapacity
+ mask = newCapacity - 1
+ }
+
+ private def nextPowerOf2(n: Int): Int = {
+ val highBit = Integer.highestOneBit(n)
+ if (highBit == n) n else highBit << 1
+ }
+}
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 a430a75451..0ce1394c77 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -17,7 +17,6 @@
package org.apache.spark.util
-import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors}
import java.util.{TimerTask, Timer}
import org.apache.spark.Logging
@@ -25,11 +24,14 @@ import org.apache.spark.Logging
/**
* Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries)
*/
-class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging {
+class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging {
+ val name = cleanerType.toString
+
private val delaySeconds = MetadataCleaner.getDelaySeconds
private val periodSeconds = math.max(10, delaySeconds / 10)
private val timer = new Timer(name + " cleanup timer", true)
+
private val task = new TimerTask {
override def run() {
try {
@@ -53,9 +55,37 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging
}
}
+object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext", "HttpBroadcast", "DagScheduler", "ResultTask",
+ "ShuffleMapTask", "BlockManager", "BroadcastVars") {
+
+ val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, SHUFFLE_MAP_TASK, BLOCK_MANAGER, BROADCAST_VARS = Value
+
+ type MetadataCleanerType = Value
+
+ def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString
+}
object MetadataCleaner {
+
+ // using only sys props for now : so that workers can also get to it while preserving earlier behavior.
def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt
- def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.ttl", delay.toString) }
+
+ def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = {
+ System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt
+ }
+
+ def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) {
+ System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString)
+ }
+
+ def setDelaySeconds(delay: Int, resetAll: Boolean = true) {
+ // override for all ?
+ System.setProperty("spark.cleaner.ttl", delay.toString)
+ if (resetAll) {
+ for (cleanerType <- MetadataCleanerType.values) {
+ System.clearProperty(MetadataCleanerType.systemProperty(cleanerType))
+ }
+ }
+ }
}
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 bb47fc0a2c..f384875cc9 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -70,6 +70,19 @@ private[spark] object Utils extends Logging {
return ois.readObject.asInstanceOf[T]
}
+ /** Deserialize a Long value (used for {@link org.apache.spark.api.python.PythonPartitioner}) */
+ def deserializeLongValue(bytes: Array[Byte]) : Long = {
+ // Note: we assume that we are given a Long value encoded in network (big-endian) byte order
+ var result = bytes(7) & 0xFFL
+ result = result + ((bytes(6) & 0xFFL) << 8)
+ result = result + ((bytes(5) & 0xFFL) << 16)
+ result = result + ((bytes(4) & 0xFFL) << 24)
+ result = result + ((bytes(3) & 0xFFL) << 32)
+ result = result + ((bytes(2) & 0xFFL) << 40)
+ result = result + ((bytes(1) & 0xFFL) << 48)
+ result + ((bytes(0) & 0xFFL) << 56)
+ }
+
/** Serialize via nested stream using specific serializer */
def serializeViaNestedStream(os: OutputStream, ser: SerializerInstance)(f: SerializationStream => Unit) = {
val osWrapper = ser.serializeStream(new OutputStream {
@@ -457,12 +470,20 @@ private[spark] object Utils extends Logging {
def newDaemonFixedThreadPool(nThreads: Int): ThreadPoolExecutor =
Executors.newFixedThreadPool(nThreads, daemonThreadFactory).asInstanceOf[ThreadPoolExecutor]
+ private def listFilesSafely(file: File): Seq[File] = {
+ val files = file.listFiles()
+ if (files == null) {
+ throw new IOException("Failed to list files for dir: " + file)
+ }
+ files
+ }
+
/**
* Delete a file or directory and its contents recursively.
*/
def deleteRecursively(file: File) {
if (file.isDirectory) {
- for (child <- file.listFiles()) {
+ for (child <- listFilesSafely(file)) {
deleteRecursively(child)
}
}
@@ -613,7 +634,7 @@ private[spark] object Utils extends Logging {
* A regular expression to match classes of the "core" Spark API that we want to skip when
* finding the call site of a method.
*/
- private val SPARK_CLASS_REGEX = """^spark(\.api\.java)?(\.rdd)?\.[A-Z]""".r
+ private val SPARK_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r
private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String,
val firstUserLine: Int, val firstUserClass: String)
@@ -778,4 +799,18 @@ private[spark] object Utils extends Logging {
val rawMod = x % mod
rawMod + (if (rawMod < 0) mod else 0)
}
+
+ // Handles idiosyncracies with hash (add more as required)
+ def nonNegativeHash(obj: AnyRef): Int = {
+
+ // Required ?
+ if (obj eq null) return 0
+
+ val hash = obj.hashCode
+ // math.abs fails for Int.MinValue
+ val hashAbs = if (Int.MinValue != hash) math.abs(hash) else 0
+
+ // Nothing else to guard against ?
+ hashAbs
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
new file mode 100644
index 0000000000..ea936e815b
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.scalatest.{BeforeAndAfter, FunSuite}
+import org.scalatest.mock.EasyMockSugar
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel}
+
+// TODO: Test the CacheManager's thread-safety aspects
+class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar {
+ var sc : SparkContext = _
+ var blockManager: BlockManager = _
+ var cacheManager: CacheManager = _
+ var split: Partition = _
+ /** An RDD which returns the values [1, 2, 3, 4]. */
+ var rdd: RDD[Int] = _
+
+ before {
+ sc = new SparkContext("local", "test")
+ blockManager = mock[BlockManager]
+ cacheManager = new CacheManager(blockManager)
+ split = new Partition { override def index: Int = 0 }
+ rdd = new RDD[Int](sc, Nil) {
+ override def getPartitions: Array[Partition] = Array(split)
+ override val getDependencies = List[Dependency[_]]()
+ override def compute(split: Partition, context: TaskContext) = Array(1, 2, 3, 4).iterator
+ }
+ }
+
+ after {
+ sc.stop()
+ }
+
+ test("get uncached rdd") {
+ expecting {
+ blockManager.get(RDDBlockId(0, 0)).andReturn(None)
+ blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY,
+ true).andReturn(0)
+ }
+
+ whenExecuting(blockManager) {
+ val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
+ taskMetrics = null)
+ val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
+ assert(value.toList === List(1, 2, 3, 4))
+ }
+ }
+
+ test("get cached rdd") {
+ expecting {
+ blockManager.get(RDDBlockId(0, 0)).andReturn(Some(ArrayBuffer(5, 6, 7).iterator))
+ }
+
+ whenExecuting(blockManager) {
+ val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false,
+ taskMetrics = null)
+ val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
+ assert(value.toList === List(5, 6, 7))
+ }
+ }
+
+ test("get uncached local rdd") {
+ expecting {
+ // Local computation should not persist the resulting value, so don't expect a put().
+ blockManager.get(RDDBlockId(0, 0)).andReturn(None)
+ }
+
+ whenExecuting(blockManager) {
+ val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false,
+ taskMetrics = null)
+ val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY)
+ assert(value.toList === List(1, 2, 3, 4))
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
index d9103aebb7..f26c44d3e7 100644
--- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
@@ -21,7 +21,7 @@ import org.scalatest.FunSuite
import java.io.File
import org.apache.spark.rdd._
import org.apache.spark.SparkContext._
-import storage.StorageLevel
+import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId}
import org.apache.spark.util.Utils
class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
@@ -62,8 +62,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
testCheckpointing(_.sample(false, 0.5, 0))
testCheckpointing(_.glom())
testCheckpointing(_.mapPartitions(_.map(_.toString)))
- testCheckpointing(r => new MapPartitionsWithIndexRDD(r,
- (i: Int, iter: Iterator[Int]) => iter.map(_.toString), false ))
+ testCheckpointing(r => new MapPartitionsWithContextRDD(r,
+ (context: TaskContext, iter: Iterator[Int]) => iter.map(_.toString), false ))
testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
testCheckpointing(_.pipe(Seq("cat")))
@@ -83,7 +83,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
}
test("BlockRDD") {
- val blockId = "id"
+ val blockId = TestBlockId("id")
val blockManager = SparkEnv.get.blockManager
blockManager.putSingle(blockId, "test", StorageLevel.MEMORY_ONLY)
val blockRDD = new BlockRDD[String](sc, Array(blockId))
@@ -191,7 +191,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
}
test("CheckpointRDD with zero partitions") {
- val rdd = new BlockRDD[Int](sc, Array[String]())
+ val rdd = new BlockRDD[Int](sc, Array[BlockId]())
assert(rdd.partitions.size === 0)
assert(rdd.isCheckpointed === false)
rdd.checkpoint()
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 7a856d4081..480bac84f3 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -18,24 +18,14 @@
package org.apache.spark
import network.ConnectionManagerId
-import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfter
import org.scalatest.concurrent.Timeouts._
+import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.prop.Checkers
import org.scalatest.time.{Span, Millis}
-import org.scalacheck.Arbitrary._
-import org.scalacheck.Gen
-import org.scalacheck.Prop._
-import org.eclipse.jetty.server.{Server, Request, Handler}
-
-import com.google.common.io.Files
-
-import scala.collection.mutable.ArrayBuffer
import SparkContext._
-import storage.{GetBlock, BlockManagerWorker, StorageLevel}
-import ui.JettyUtils
+import org.apache.spark.storage.{BlockManagerWorker, GetBlock, RDDBlockId, StorageLevel}
class NotSerializableClass
@@ -193,7 +183,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
// Get all the locations of the first partition and try to fetch the partitions
// from those locations.
- val blockIds = data.partitions.indices.map(index => "rdd_%d_%d".format(data.id, index)).toArray
+ val blockIds = data.partitions.indices.map(index => RDDBlockId(data.id, index)).toArray
val blockId = blockIds(0)
val blockManager = SparkEnv.get.blockManager
blockManager.master.getLocations(blockId).foreach(id => {
@@ -319,19 +309,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.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/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
index 8a869c9005..7b0bb89ab2 100644
--- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
@@ -495,7 +495,7 @@ public class JavaAPISuite implements Serializable {
@Test
public void iterator() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2);
- TaskContext context = new TaskContext(0, 0, 0, null);
+ TaskContext context = new TaskContext(0, 0, 0, false, false, null);
Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue());
}
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
new file mode 100644
index 0000000000..a192651491
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+import java.util.concurrent.Semaphore
+
+import scala.concurrent.future
+import scala.concurrent.ExecutionContext.Implicits.global
+
+import org.scalatest.{BeforeAndAfter, FunSuite}
+import org.scalatest.matchers.ShouldMatchers
+
+import org.apache.spark.SparkContext._
+import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListener}
+
+
+/**
+ * Test suite for cancelling running jobs. We run the cancellation tasks for single job action
+ * (e.g. count) as well as multi-job action (e.g. take). We test the local and cluster schedulers
+ * in both FIFO and fair scheduling modes.
+ */
+class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
+ with LocalSparkContext {
+
+ override def afterEach() {
+ super.afterEach()
+ resetSparkContext()
+ System.clearProperty("spark.scheduler.mode")
+ }
+
+ test("local mode, FIFO scheduler") {
+ System.setProperty("spark.scheduler.mode", "FIFO")
+ sc = new SparkContext("local[2]", "test")
+ testCount()
+ testTake()
+ // Make sure we can still launch tasks.
+ assert(sc.parallelize(1 to 10, 2).count === 10)
+ }
+
+ test("local mode, fair scheduler") {
+ System.setProperty("spark.scheduler.mode", "FAIR")
+ val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
+ System.setProperty("spark.scheduler.allocation.file", xmlPath)
+ sc = new SparkContext("local[2]", "test")
+ testCount()
+ testTake()
+ // Make sure we can still launch tasks.
+ assert(sc.parallelize(1 to 10, 2).count === 10)
+ }
+
+ test("cluster mode, FIFO scheduler") {
+ System.setProperty("spark.scheduler.mode", "FIFO")
+ sc = new SparkContext("local-cluster[2,1,512]", "test")
+ testCount()
+ testTake()
+ // Make sure we can still launch tasks.
+ assert(sc.parallelize(1 to 10, 2).count === 10)
+ }
+
+ test("cluster mode, fair scheduler") {
+ System.setProperty("spark.scheduler.mode", "FAIR")
+ val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
+ System.setProperty("spark.scheduler.allocation.file", xmlPath)
+ sc = new SparkContext("local-cluster[2,1,512]", "test")
+ testCount()
+ testTake()
+ // Make sure we can still launch tasks.
+ assert(sc.parallelize(1 to 10, 2).count === 10)
+ }
+
+ test("two jobs sharing the same stage") {
+ // sem1: make sure cancel is issued after some tasks are launched
+ // sem2: make sure the first stage is not finished until cancel is issued
+ val sem1 = new Semaphore(0)
+ val sem2 = new Semaphore(0)
+
+ sc = new SparkContext("local[2]", "test")
+ sc.dagScheduler.addSparkListener(new SparkListener {
+ override def onTaskStart(taskStart: SparkListenerTaskStart) {
+ sem1.release()
+ }
+ })
+
+ // Create two actions that would share the some stages.
+ val rdd = sc.parallelize(1 to 10, 2).map { i =>
+ sem2.acquire()
+ (i, i)
+ }.reduceByKey(_+_)
+ val f1 = rdd.collectAsync()
+ val f2 = rdd.countAsync()
+
+ // Kill one of the action.
+ future {
+ sem1.acquire()
+ f1.cancel()
+ sem2.release(10)
+ }
+
+ // Expect both to fail now.
+ // TODO: update this test when we change Spark so cancelling f1 wouldn't affect f2.
+ intercept[SparkException] { f1.get() }
+ intercept[SparkException] { f2.get() }
+ }
+
+ def testCount() {
+ // Cancel before launching any tasks
+ {
+ val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.countAsync()
+ future { f.cancel() }
+ val e = intercept[SparkException] { f.get() }
+ assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+ }
+
+ // Cancel after some tasks have been launched
+ {
+ // Add a listener to release the semaphore once any tasks are launched.
+ val sem = new Semaphore(0)
+ sc.dagScheduler.addSparkListener(new SparkListener {
+ override def onTaskStart(taskStart: SparkListenerTaskStart) {
+ sem.release()
+ }
+ })
+
+ val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.countAsync()
+ future {
+ // Wait until some tasks were launched before we cancel the job.
+ sem.acquire()
+ f.cancel()
+ }
+ val e = intercept[SparkException] { f.get() }
+ assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+ }
+ }
+
+ def testTake() {
+ // Cancel before launching any tasks
+ {
+ val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.takeAsync(5000)
+ future { f.cancel() }
+ val e = intercept[SparkException] { f.get() }
+ assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+ }
+
+ // Cancel after some tasks have been launched
+ {
+ // Add a listener to release the semaphore once any tasks are launched.
+ val sem = new Semaphore(0)
+ sc.dagScheduler.addSparkListener(new SparkListener {
+ override def onTaskStart(taskStart: SparkListenerTaskStart) {
+ sem.release()
+ }
+ })
+ val f = sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.takeAsync(5000)
+ future {
+ sem.acquire()
+ f.cancel()
+ }
+ val e = intercept[SparkException] { f.get() }
+ assert(e.getMessage.contains("cancelled") || e.getMessage.contains("killed"))
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
index 6ec124da9c..459e257d79 100644
--- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala
@@ -40,17 +40,17 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self
}
def resetSparkContext() = {
- if (sc != null) {
- LocalSparkContext.stop(sc)
- sc = null
- }
+ LocalSparkContext.stop(sc)
+ sc = null
}
}
object LocalSparkContext {
def stop(sc: SparkContext) {
- sc.stop()
+ if (sc != null) {
+ sc.stop()
+ }
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
index 97cbca09bf..288aa14eeb 100644
--- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
@@ -33,10 +33,8 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite =>
}
override def afterAll() {
- if (_sc != null) {
- LocalSparkContext.stop(_sc)
- _sc = null
- }
+ LocalSparkContext.stop(_sc)
+ _sc = null
super.afterAll()
}
}
diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
index 69383ddfb8..75d6493e33 100644
--- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala
@@ -40,7 +40,7 @@ object ThreadingSuiteState {
}
class ThreadingSuite extends FunSuite with LocalSparkContext {
-
+
test("accessing SparkContext form a different thread") {
sc = new SparkContext("local", "test")
val nums = sc.parallelize(1 to 10, 2)
@@ -149,4 +149,47 @@ class ThreadingSuite extends FunSuite with LocalSparkContext {
fail("One or more threads didn't see runningThreads = 4")
}
}
+
+ test("set local properties in different thread") {
+ sc = new SparkContext("local", "test")
+ val sem = new Semaphore(0)
+
+ val threads = (1 to 5).map { i =>
+ new Thread() {
+ override def run() {
+ sc.setLocalProperty("test", i.toString)
+ assert(sc.getLocalProperty("test") === i.toString)
+ sem.release()
+ }
+ }
+ }
+
+ threads.foreach(_.start())
+
+ sem.acquire(5)
+ assert(sc.getLocalProperty("test") === null)
+ }
+
+ test("set and get local properties in parent-children thread") {
+ sc = new SparkContext("local", "test")
+ sc.setLocalProperty("test", "parent")
+ val sem = new Semaphore(0)
+
+ val threads = (1 to 5).map { i =>
+ new Thread() {
+ override def run() {
+ assert(sc.getLocalProperty("test") === "parent")
+ sc.setLocalProperty("test", i.toString)
+ assert(sc.getLocalProperty("test") === i.toString)
+ sem.release()
+ }
+ }
+ }
+
+ threads.foreach(_.start())
+
+ sem.acquire(5)
+ assert(sc.getLocalProperty("test") === "parent")
+ assert(sc.getLocalProperty("Foo") === null)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
new file mode 100644
index 0000000000..0b38e239f9
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -0,0 +1,93 @@
+/*
+ * 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
+
+import java.io.File
+import java.util.Date
+
+import net.liftweb.json.{JsonAST, JsonParser}
+import net.liftweb.json.JsonAST.JValue
+import org.scalatest.FunSuite
+
+import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse}
+import org.apache.spark.deploy.master.{ApplicationInfo, RecoveryState, WorkerInfo}
+import org.apache.spark.deploy.worker.ExecutorRunner
+
+class JsonProtocolSuite extends FunSuite {
+ test("writeApplicationInfo") {
+ val output = JsonProtocol.writeApplicationInfo(createAppInfo())
+ assertValidJson(output)
+ }
+
+ test("writeWorkerInfo") {
+ val output = JsonProtocol.writeWorkerInfo(createWorkerInfo())
+ assertValidJson(output)
+ }
+
+ test("writeApplicationDescription") {
+ val output = JsonProtocol.writeApplicationDescription(createAppDesc())
+ assertValidJson(output)
+ }
+
+ test("writeExecutorRunner") {
+ val output = JsonProtocol.writeExecutorRunner(createExecutorRunner())
+ assertValidJson(output)
+ }
+
+ test("writeMasterState") {
+ val workers = Array[WorkerInfo](createWorkerInfo(), createWorkerInfo())
+ val activeApps = Array[ApplicationInfo](createAppInfo())
+ val completedApps = Array[ApplicationInfo]()
+ val stateResponse = new MasterStateResponse("host", 8080, workers, activeApps, completedApps,
+ RecoveryState.ALIVE)
+ val output = JsonProtocol.writeMasterState(stateResponse)
+ assertValidJson(output)
+ }
+
+ test("writeWorkerState") {
+ val executors = List[ExecutorRunner]()
+ val finishedExecutors = List[ExecutorRunner](createExecutorRunner(), createExecutorRunner())
+ val stateResponse = new WorkerStateResponse("host", 8080, "workerId", executors,
+ finishedExecutors, "masterUrl", 4, 1234, 4, 1234, "masterWebUiUrl")
+ val output = JsonProtocol.writeWorkerState(stateResponse)
+ assertValidJson(output)
+ }
+
+ def createAppDesc() : ApplicationDescription = {
+ val cmd = new Command("mainClass", List("arg1", "arg2"), Map())
+ new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl")
+ }
+ def createAppInfo() : ApplicationInfo = {
+ new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr")
+ }
+ def createWorkerInfo() : WorkerInfo = {
+ new WorkerInfo("id", "host", 8080, 4, 1234, null, 80, "publicAddress")
+ }
+ def createExecutorRunner() : ExecutorRunner = {
+ new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host",
+ new File("sparkHome"), new File("workDir"), ExecutorState.RUNNING)
+ }
+
+ def assertValidJson(json: JValue) {
+ try {
+ JsonParser.parse(JsonAST.compactRender(json))
+ } catch {
+ case e: JsonParser.ParseException => fail("Invalid Json detected", e)
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
index 58c94a162d..1a9ce8c607 100644
--- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala
@@ -30,14 +30,13 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
val conf = new MetricsConfig(Option("dummy-file"))
conf.initialize()
- assert(conf.properties.size() === 5)
+ assert(conf.properties.size() === 4)
assert(conf.properties.getProperty("test-for-dummy") === null)
val property = conf.getInstance("random")
- assert(property.size() === 3)
+ assert(property.size() === 2)
assert(property.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet")
- assert(property.getProperty("sink.servlet.uri") === "/metrics/json")
- assert(property.getProperty("sink.servlet.sample") === "false")
+ assert(property.getProperty("sink.servlet.path") === "/metrics/json")
}
test("MetricsConfig with properties set") {
@@ -45,22 +44,20 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
conf.initialize()
val masterProp = conf.getInstance("master")
- assert(masterProp.size() === 6)
+ assert(masterProp.size() === 5)
assert(masterProp.getProperty("sink.console.period") === "20")
assert(masterProp.getProperty("sink.console.unit") === "minutes")
assert(masterProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource")
assert(masterProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet")
- assert(masterProp.getProperty("sink.servlet.uri") === "/metrics/master/json")
- assert(masterProp.getProperty("sink.servlet.sample") === "false")
+ assert(masterProp.getProperty("sink.servlet.path") === "/metrics/master/json")
val workerProp = conf.getInstance("worker")
- assert(workerProp.size() === 6)
+ assert(workerProp.size() === 5)
assert(workerProp.getProperty("sink.console.period") === "10")
assert(workerProp.getProperty("sink.console.unit") === "seconds")
assert(workerProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource")
assert(workerProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet")
- assert(workerProp.getProperty("sink.servlet.uri") === "/metrics/json")
- assert(workerProp.getProperty("sink.servlet.sample") === "false")
+ assert(workerProp.getProperty("sink.servlet.path") === "/metrics/json")
}
test("MetricsConfig with subProperties") {
@@ -84,6 +81,6 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter {
assert(consoleProps.size() === 2)
val servletProps = sinkProps("servlet")
- assert(servletProps.size() === 3)
+ assert(servletProps.size() === 2)
}
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
new file mode 100644
index 0000000000..da032b17d9
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
@@ -0,0 +1,176 @@
+/*
+ * 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.rdd
+
+import java.util.concurrent.Semaphore
+
+import scala.concurrent.ExecutionContext.Implicits.global
+
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
+import org.scalatest.concurrent.Timeouts
+import org.scalatest.time.SpanSugar._
+
+import org.apache.spark.SparkContext._
+import org.apache.spark.{SparkContext, SparkException, LocalSparkContext}
+
+
+class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts {
+
+ @transient private var sc: SparkContext = _
+
+ override def beforeAll() {
+ sc = new SparkContext("local[2]", "test")
+ }
+
+ override def afterAll() {
+ LocalSparkContext.stop(sc)
+ sc = null
+ }
+
+ lazy val zeroPartRdd = new EmptyRDD[Int](sc)
+
+ test("countAsync") {
+ assert(zeroPartRdd.countAsync().get() === 0)
+ assert(sc.parallelize(1 to 10000, 5).countAsync().get() === 10000)
+ }
+
+ test("collectAsync") {
+ assert(zeroPartRdd.collectAsync().get() === Seq.empty)
+
+ val collected = sc.parallelize(1 to 1000, 3).collectAsync().get()
+ assert(collected === (1 to 1000))
+ }
+
+ test("foreachAsync") {
+ zeroPartRdd.foreachAsync(i => Unit).get()
+
+ val accum = sc.accumulator(0)
+ sc.parallelize(1 to 1000, 3).foreachAsync { i =>
+ accum += 1
+ }.get()
+ assert(accum.value === 1000)
+ }
+
+ test("foreachPartitionAsync") {
+ zeroPartRdd.foreachPartitionAsync(iter => Unit).get()
+
+ val accum = sc.accumulator(0)
+ sc.parallelize(1 to 1000, 9).foreachPartitionAsync { iter =>
+ accum += 1
+ }.get()
+ assert(accum.value === 9)
+ }
+
+ test("takeAsync") {
+ def testTake(rdd: RDD[Int], input: Seq[Int], num: Int) {
+ val expected = input.take(num)
+ val saw = rdd.takeAsync(num).get()
+ assert(saw == expected, "incorrect result for rdd with %d partitions (expected %s, saw %s)"
+ .format(rdd.partitions.size, expected, saw))
+ }
+ val input = Range(1, 1000)
+
+ var rdd = sc.parallelize(input, 1)
+ for (num <- Seq(0, 1, 999, 1000)) {
+ testTake(rdd, input, num)
+ }
+
+ rdd = sc.parallelize(input, 2)
+ for (num <- Seq(0, 1, 3, 500, 501, 999, 1000)) {
+ testTake(rdd, input, num)
+ }
+
+ rdd = sc.parallelize(input, 100)
+ for (num <- Seq(0, 1, 500, 501, 999, 1000)) {
+ testTake(rdd, input, num)
+ }
+
+ rdd = sc.parallelize(input, 1000)
+ for (num <- Seq(0, 1, 3, 999, 1000)) {
+ testTake(rdd, input, num)
+ }
+ }
+
+ /**
+ * Make sure onComplete, onSuccess, and onFailure are invoked correctly in the case
+ * of a successful job execution.
+ */
+ test("async success handling") {
+ val f = sc.parallelize(1 to 10, 2).countAsync()
+
+ // Use a semaphore to make sure onSuccess and onComplete's success path will be called.
+ // If not, the test will hang.
+ val sem = new Semaphore(0)
+
+ f.onComplete {
+ case scala.util.Success(res) =>
+ sem.release()
+ case scala.util.Failure(e) =>
+ info("Should not have reached this code path (onComplete matching Failure)")
+ throw new Exception("Task should succeed")
+ }
+ f.onSuccess { case a: Any =>
+ sem.release()
+ }
+ f.onFailure { case t =>
+ info("Should not have reached this code path (onFailure)")
+ throw new Exception("Task should succeed")
+ }
+ assert(f.get() === 10)
+
+ failAfter(10 seconds) {
+ sem.acquire(2)
+ }
+ }
+
+ /**
+ * Make sure onComplete, onSuccess, and onFailure are invoked correctly in the case
+ * of a failed job execution.
+ */
+ test("async failure handling") {
+ val f = sc.parallelize(1 to 10, 2).map { i =>
+ throw new Exception("intentional"); i
+ }.countAsync()
+
+ // Use a semaphore to make sure onFailure and onComplete's failure path will be called.
+ // If not, the test will hang.
+ val sem = new Semaphore(0)
+
+ f.onComplete {
+ case scala.util.Success(res) =>
+ info("Should not have reached this code path (onComplete matching Success)")
+ throw new Exception("Task should fail")
+ case scala.util.Failure(e) =>
+ sem.release()
+ }
+ f.onSuccess { case a: Any =>
+ info("Should not have reached this code path (onSuccess)")
+ throw new Exception("Task should fail")
+ }
+ f.onFailure { case t =>
+ sem.release()
+ }
+ intercept[SparkException] {
+ f.get()
+ }
+
+ failAfter(10 seconds) {
+ sem.acquire(2)
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
index 31f97fc139..57d3382ed0 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
@@ -106,7 +106,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext {
}
}
visit(sums)
- assert(deps.size === 2) // ShuffledRDD, ParallelCollection
+ assert(deps.size === 2) // ShuffledRDD, ParallelCollection.
}
test("join") {
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 adc971050e..6d1bc5e296 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -25,7 +25,6 @@ import org.apache.spark.SparkContext._
import org.apache.spark.rdd._
import scala.collection.parallel.mutable
import org.apache.spark._
-import org.apache.spark.rdd.CoalescedRDDPartition
class RDDSuite extends FunSuite with SharedSparkContext {
@@ -140,7 +139,7 @@ class RDDSuite extends FunSuite with SharedSparkContext {
assert(rdd.union(emptyKv).collect().size === 2)
}
- test("cogrouped RDDs") {
+ test("coalesced RDDs") {
val data = sc.parallelize(1 to 10, 10)
val coalesced1 = data.coalesce(2)
@@ -175,8 +174,14 @@ class RDDSuite extends FunSuite with SharedSparkContext {
val coalesced5 = data.coalesce(1, shuffle = true)
assert(coalesced5.dependencies.head.rdd.dependencies.head.rdd.asInstanceOf[ShuffledRDD[_, _, _]] !=
null)
+
+ // when shuffling, we can increase the number of partitions
+ val coalesced6 = data.coalesce(20, shuffle = true)
+ assert(coalesced6.partitions.size === 20)
+ assert(coalesced6.collect().toSet === (1 to 10).toSet)
}
- test("cogrouped RDDs with locality") {
+
+ test("coalesced RDDs with locality") {
val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b"))))
val coal3 = data3.coalesce(3)
val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation)
@@ -197,11 +202,11 @@ class RDDSuite extends FunSuite with SharedSparkContext {
val coalesced4 = data.coalesce(20)
val listOfLists = coalesced4.glom().collect().map(_.toList).toList
val sortedList = listOfLists.sortWith{ (x, y) => !x.isEmpty && (y.isEmpty || (x(0) < y(0))) }
- assert( sortedList === (1 to 9).
+ assert(sortedList === (1 to 9).
map{x => List(x)}.toList, "Tried coalescing 9 partitions to 20 but didn't get 9 back")
}
- test("cogrouped RDDs with locality, large scale (10K partitions)") {
+ test("coalesced RDDs with locality, large scale (10K partitions)") {
// large scale experiment
import collection.mutable
val rnd = scala.util.Random
@@ -315,6 +320,44 @@ class RDDSuite extends FunSuite with SharedSparkContext {
for (i <- 0 until sample.size) assert(sample(i) === checkSample(i))
}
+ test("take") {
+ var nums = sc.makeRDD(Range(1, 1000), 1)
+ assert(nums.take(0).size === 0)
+ assert(nums.take(1) === Array(1))
+ assert(nums.take(3) === Array(1, 2, 3))
+ assert(nums.take(500) === (1 to 500).toArray)
+ assert(nums.take(501) === (1 to 501).toArray)
+ assert(nums.take(999) === (1 to 999).toArray)
+ assert(nums.take(1000) === (1 to 999).toArray)
+
+ nums = sc.makeRDD(Range(1, 1000), 2)
+ assert(nums.take(0).size === 0)
+ assert(nums.take(1) === Array(1))
+ assert(nums.take(3) === Array(1, 2, 3))
+ assert(nums.take(500) === (1 to 500).toArray)
+ assert(nums.take(501) === (1 to 501).toArray)
+ assert(nums.take(999) === (1 to 999).toArray)
+ assert(nums.take(1000) === (1 to 999).toArray)
+
+ nums = sc.makeRDD(Range(1, 1000), 100)
+ assert(nums.take(0).size === 0)
+ assert(nums.take(1) === Array(1))
+ assert(nums.take(3) === Array(1, 2, 3))
+ assert(nums.take(500) === (1 to 500).toArray)
+ assert(nums.take(501) === (1 to 501).toArray)
+ assert(nums.take(999) === (1 to 999).toArray)
+ assert(nums.take(1000) === (1 to 999).toArray)
+
+ nums = sc.makeRDD(Range(1, 1000), 1000)
+ assert(nums.take(0).size === 0)
+ assert(nums.take(1) === Array(1))
+ assert(nums.take(3) === Array(1, 2, 3))
+ assert(nums.take(500) === (1 to 500).toArray)
+ assert(nums.take(501) === (1 to 501).toArray)
+ assert(nums.take(999) === (1 to 999).toArray)
+ assert(nums.take(1000) === (1 to 999).toArray)
+ }
+
test("top with predefined ordering") {
val nums = Array.range(1, 100000)
val ints = sc.makeRDD(scala.util.Random.shuffle(nums), 2)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
index 94f66c94c6..838179c6b5 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -24,17 +24,14 @@ import org.scalatest.BeforeAndAfter
import org.apache.spark.LocalSparkContext
import org.apache.spark.MapOutputTracker
-import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext
import org.apache.spark.Partition
import org.apache.spark.TaskContext
import org.apache.spark.{Dependency, ShuffleDependency, OneToOneDependency}
import org.apache.spark.{FetchFailed, Success, TaskEndReason}
-import org.apache.spark.storage.{BlockManagerId, BlockManagerMaster}
-
-import org.apache.spark.scheduler.cluster.Pool
-import org.apache.spark.scheduler.cluster.SchedulingMode
-import org.apache.spark.scheduler.cluster.SchedulingMode.SchedulingMode
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
+import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
/**
* Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler
@@ -62,6 +59,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch)
taskSets += taskSet
}
+ override def cancelTasks(stageId: Int) {}
override def setListener(listener: TaskSchedulerListener) = {}
override def defaultParallelism() = 2
}
@@ -77,15 +75,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]]
// stub out BlockManagerMaster.getLocations to use our cacheLocations
val blockManagerMaster = new BlockManagerMaster(null) {
- override def getLocations(blockIds: Array[String]): Seq[Seq[BlockManagerId]] = {
- blockIds.map { name =>
- val pieces = name.split("_")
- if (pieces(0) == "rdd") {
- val key = pieces(1).toInt -> pieces(2).toInt
- cacheLocations.getOrElse(key, Seq())
- } else {
- Seq()
- }
+ override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
+ blockIds.map {
+ _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)).
+ getOrElse(Seq())
}.toSeq
}
override def removeExecutor(execId: String) {
@@ -188,7 +181,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
func: (TaskContext, Iterator[_]) => _ = jobComputeFunc,
allowLocal: Boolean = false,
listener: JobListener = listener) {
- runEvent(JobSubmitted(rdd, func, partitions, allowLocal, null, listener))
+ val jobId = scheduler.nextJobId.getAndIncrement()
+ runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener))
}
/** Sends TaskSetFailed to the scheduler. */
@@ -222,7 +216,8 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
override def getPreferredLocations(split: Partition) = Nil
override def toString = "DAGSchedulerSuite Local RDD"
}
- runEvent(JobSubmitted(rdd, jobComputeFunc, Array(0), true, null, listener))
+ val jobId = scheduler.nextJobId.getAndIncrement()
+ runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener))
assert(results === Map(0 -> 42))
}
@@ -249,7 +244,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
test("trivial job failure") {
submit(makeRdd(1, Nil), Array(0))
failed(taskSets(0), "some failure")
- assert(failure.getMessage === "Job failed: some failure")
+ assert(failure.getMessage === "Job aborted: some failure")
}
test("run trivial shuffle") {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index aac7c207cb..a549417a47 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -23,10 +23,6 @@ import scala.collection.mutable
import org.scalatest.matchers.ShouldMatchers
import org.apache.spark.SparkContext._
-/**
- *
- */
-
class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers {
test("local metrics") {
@@ -42,7 +38,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
}
val d = sc.parallelize(1 to 1e4.toInt, 64).map{i => w(i)}
- d.count
+ d.count()
+ val WAIT_TIMEOUT_MILLIS = 10000
+ assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
listener.stageInfos.size should be (1)
val d2 = d.map{i => w(i) -> i * 2}.setName("shuffle input 1")
@@ -52,19 +50,27 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)}
d4.setName("A Cogroup")
- d4.collectAsMap
+ d4.collectAsMap()
+ assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
listener.stageInfos.size should be (4)
- listener.stageInfos.foreach {stageInfo =>
- //small test, so some tasks might take less than 1 millisecond, but average should be greater than 1 ms
+ listener.stageInfos.foreach { stageInfo =>
+ /* small test, so some tasks might take less than 1 millisecond, but average should be greater
+ * than 0 ms. */
checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration")
- checkNonZeroAvg(stageInfo.taskInfos.map{_._2.executorRunTime.toLong}, stageInfo + " executorRunTime")
- checkNonZeroAvg(stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong}, stageInfo + " executorDeserializeTime")
+ checkNonZeroAvg(
+ stageInfo.taskInfos.map{_._2.executorRunTime.toLong},
+ stageInfo + " executorRunTime")
+ checkNonZeroAvg(
+ stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong},
+ stageInfo + " executorDeserializeTime")
if (stageInfo.stage.rdd.name == d4.name) {
- checkNonZeroAvg(stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, stageInfo + " fetchWaitTime")
+ checkNonZeroAvg(
+ stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime},
+ stageInfo + " fetchWaitTime")
}
- stageInfo.taskInfos.foreach{case (taskInfo, taskMetrics) =>
+ stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) =>
taskMetrics.resultSize should be > (0l)
if (isStage(stageInfo, Set(d2.name, d3.name), Set(d4.name))) {
taskMetrics.shuffleWriteMetrics should be ('defined)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
index 92ad9f09b2..95d3553d91 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
@@ -43,16 +43,16 @@ class FakeTaskSetManager(
stageId = initStageId
name = "TaskSet_"+stageId
override val numTasks = initNumTasks
- tasksFinished = 0
+ tasksSuccessful = 0
- override def increaseRunningTasks(taskNum: Int) {
+ def increaseRunningTasks(taskNum: Int) {
runningTasks += taskNum
if (parent != null) {
parent.increaseRunningTasks(taskNum)
}
}
- override def decreaseRunningTasks(taskNum: Int) {
+ def decreaseRunningTasks(taskNum: Int) {
runningTasks -= taskNum
if (parent != null) {
parent.decreaseRunningTasks(taskNum)
@@ -79,7 +79,7 @@ class FakeTaskSetManager(
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription] =
{
- if (tasksFinished + runningTasks < numTasks) {
+ if (tasksSuccessful + runningTasks < numTasks) {
increaseRunningTasks(1)
return Some(new TaskDescription(0, execId, "task 0:0", 0, null))
}
@@ -92,8 +92,8 @@ class FakeTaskSetManager(
def taskFinished() {
decreaseRunningTasks(1)
- tasksFinished +=1
- if (tasksFinished == numTasks) {
+ tasksSuccessful +=1
+ if (tasksSuccessful == numTasks) {
parent.removeSchedulable(this)
}
}
@@ -114,7 +114,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
val taskSetQueue = rootPool.getSortedTaskSetQueue()
/* Just for Test*/
for (manager <- taskSetQueue) {
- logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks))
+ logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format(
+ manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks))
}
for (taskSet <- taskSetQueue) {
taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match {
@@ -166,7 +167,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
val taskSet = new TaskSet(tasks.toArray,0,0,0,null)
val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
- System.setProperty("spark.fairscheduler.allocation.file", xmlPath)
+ System.setProperty("spark.scheduler.allocation.file", xmlPath)
val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
val schedulableBuilder = new FairSchedulableBuilder(rootPool)
schedulableBuilder.buildPools()
@@ -179,13 +180,13 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
assert(rootPool.getSchedulableByName("1").weight === 1)
assert(rootPool.getSchedulableByName("2").minShare === 3)
assert(rootPool.getSchedulableByName("2").weight === 1)
- assert(rootPool.getSchedulableByName("3").minShare === 2)
+ assert(rootPool.getSchedulableByName("3").minShare === 0)
assert(rootPool.getSchedulableByName("3").weight === 1)
val properties1 = new Properties()
- properties1.setProperty("spark.scheduler.cluster.fair.pool","1")
+ properties1.setProperty("spark.scheduler.pool","1")
val properties2 = new Properties()
- properties2.setProperty("spark.scheduler.cluster.fair.pool","2")
+ properties2.setProperty("spark.scheduler.pool","2")
val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet)
val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet)
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 ff70a2cdf0..80d0c5a5e9 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
@@ -40,6 +40,7 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /*
val startedTasks = new ArrayBuffer[Long]
val endedTasks = new mutable.HashMap[Long, TaskEndReason]
val finishedManagers = new ArrayBuffer[TaskSetManager]
+ val taskSetsFailed = new ArrayBuffer[String]
val executors = new mutable.HashMap[String, String] ++ liveExecutors
@@ -63,7 +64,9 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /*
def executorLost(execId: String) {}
- def taskSetFailed(taskSet: TaskSet, reason: String) {}
+ def taskSetFailed(taskSet: TaskSet, reason: String) {
+ taskSetsFailed += taskSet.id
+ }
}
def removeExecutor(execId: String): Unit = executors -= execId
@@ -101,7 +104,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
assert(manager.resourceOffer("exec1", "host1", 2, PROCESS_LOCAL) === None)
// Tell it the task has finished
- manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0))
+ manager.handleSuccessfulTask(0, createTaskResult(0))
assert(sched.endedTasks(0) === Success)
assert(sched.finishedManagers.contains(manager))
}
@@ -125,14 +128,14 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
// Finish the first two tasks
- manager.statusUpdate(0, TaskState.FINISHED, createTaskResult(0))
- manager.statusUpdate(1, TaskState.FINISHED, createTaskResult(1))
+ manager.handleSuccessfulTask(0, createTaskResult(0))
+ manager.handleSuccessfulTask(1, createTaskResult(1))
assert(sched.endedTasks(0) === Success)
assert(sched.endedTasks(1) === Success)
assert(!sched.finishedManagers.contains(manager))
// Finish the last task
- manager.statusUpdate(2, TaskState.FINISHED, createTaskResult(2))
+ manager.handleSuccessfulTask(2, createTaskResult(2))
assert(sched.endedTasks(2) === Success)
assert(sched.finishedManagers.contains(manager))
}
@@ -253,6 +256,47 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None)
}
+ test("task result lost") {
+ sc = new SparkContext("local", "test")
+ val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
+ val taskSet = createTaskSet(1)
+ val clock = new FakeClock
+ val manager = new ClusterTaskSetManager(sched, taskSet, clock)
+
+ assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+
+ // Tell it the task has finished but the result was lost.
+ manager.handleFailedTask(0, TaskState.FINISHED, Some(TaskResultLost))
+ assert(sched.endedTasks(0) === TaskResultLost)
+
+ // Re-offer the host -- now we should get task 0 again.
+ assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ }
+
+ test("repeated failures lead to task set abortion") {
+ sc = new SparkContext("local", "test")
+ val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
+ val taskSet = createTaskSet(1)
+ val clock = new FakeClock
+ val manager = new ClusterTaskSetManager(sched, taskSet, clock)
+
+ // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted
+ // after the last failure.
+ (0 until manager.MAX_TASK_FAILURES).foreach { index =>
+ val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY)
+ assert(offerResult != None,
+ "Expect resource offer on iteration %s to return a task".format(index))
+ assert(offerResult.get.index === 0)
+ manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, Some(TaskResultLost))
+ if (index < manager.MAX_TASK_FAILURES) {
+ assert(!sched.taskSetsFailed.contains(taskSet.id))
+ } else {
+ assert(sched.taskSetsFailed.contains(taskSet.id))
+ }
+ }
+ }
+
+
/**
* Utility method to create a TaskSet, potentially setting a particular sequence of preferred
* locations for each task (given as varargs) if this sequence is not empty.
@@ -267,7 +311,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
new TaskSet(tasks, 0, 0, 0, null)
}
- def createTaskResult(id: Int): ByteBuffer = {
- ByteBuffer.wrap(Utils.serialize(new TaskResult[Int](id, mutable.Map.empty, new TaskMetrics)))
+ def createTaskResult(id: Int): DirectTaskResult[Int] = {
+ new DirectTaskResult[Int](id, mutable.Map.empty, new TaskMetrics)
}
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
index 2f12aaed18..0f01515179 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
@@ -17,10 +17,11 @@
package org.apache.spark.scheduler.cluster
+import org.apache.spark.TaskContext
import org.apache.spark.scheduler.{TaskLocation, Task}
-class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId) {
- override def run(attemptId: Long): Int = 0
+class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0) {
+ override def runTask(context: TaskContext): Int = 0
override def preferredLocations: Seq[TaskLocation] = prefLocs
}
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
new file mode 100644
index 0000000000..ee150a3107
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster
+
+import java.nio.ByteBuffer
+
+import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
+
+import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv}
+import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, TaskResult}
+import org.apache.spark.storage.TaskResultBlockId
+
+/**
+ * Removes the TaskResult from the BlockManager before delegating to a normal TaskResultGetter.
+ *
+ * Used to test the case where a BlockManager evicts the task result (or dies) before the
+ * TaskResult is retrieved.
+ */
+class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler)
+ extends TaskResultGetter(sparkEnv, scheduler) {
+ var removedResult = false
+
+ override def enqueueSuccessfulTask(
+ taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) {
+ if (!removedResult) {
+ // Only remove the result once, since we'd like to test the case where the task eventually
+ // succeeds.
+ serializer.get().deserialize[TaskResult[_]](serializedData) match {
+ case IndirectTaskResult(blockId) =>
+ sparkEnv.blockManager.master.removeBlock(blockId)
+ case directResult: DirectTaskResult[_] =>
+ taskSetManager.abort("Internal error: expect only indirect results")
+ }
+ serializedData.rewind()
+ removedResult = true
+ }
+ super.enqueueSuccessfulTask(taskSetManager, tid, serializedData)
+ }
+}
+
+/**
+ * Tests related to handling task results (both direct and indirect).
+ */
+class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll
+ with LocalSparkContext {
+
+ override def beforeAll {
+ // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small
+ // as we can make it) so the tests don't take too long.
+ System.setProperty("spark.akka.frameSize", "1")
+ }
+
+ before {
+ // Use local-cluster mode because results are returned differently when running with the
+ // LocalScheduler.
+ sc = new SparkContext("local-cluster[1,1,512]", "test")
+ }
+
+ override def afterAll {
+ System.clearProperty("spark.akka.frameSize")
+ }
+
+ test("handling results smaller than Akka frame size") {
+ val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x)
+ assert(result === 2)
+ }
+
+ test("handling results larger than Akka frame size") {
+ val akkaFrameSize =
+ sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-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)
+
+ val RESULT_BLOCK_ID = TaskResultBlockId(0)
+ assert(sc.env.blockManager.master.getLocations(RESULT_BLOCK_ID).size === 0,
+ "Expect result to be removed from the block manager.")
+ }
+
+ test("task retried if result missing from block manager") {
+ // If this test hangs, it's probably because no resource offers were made after the task
+ // failed.
+ val scheduler: ClusterScheduler = sc.taskScheduler match {
+ case clusterScheduler: ClusterScheduler =>
+ clusterScheduler
+ case _ =>
+ assert(false, "Expect local cluster to use ClusterScheduler")
+ throw new ClassCastException
+ }
+ scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler)
+ val akkaFrameSize =
+ sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-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)
+
+ // Make sure two tasks were run (one failed one, and a second retried one).
+ assert(scheduler.nextTaskId.get() === 2)
+ }
+}
+
diff --git a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
index 111340a65c..1e676c1719 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
@@ -17,17 +17,15 @@
package org.apache.spark.scheduler.local
-import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
-
-import org.apache.spark._
-import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.cluster._
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.{ConcurrentMap, HashMap}
import java.util.concurrent.Semaphore
import java.util.concurrent.CountDownLatch
-import java.util.Properties
+
+import scala.collection.mutable.HashMap
+
+import org.scalatest.{BeforeAndAfterEach, FunSuite}
+
+import org.apache.spark._
+
class Lock() {
var finished = false
@@ -63,7 +61,12 @@ object TaskThreadInfo {
* 5. each task(pending) must use "sleep" to make sure it has been added to taskSetManager queue,
* thus it will be scheduled later when cluster has free cpu cores.
*/
-class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
+class LocalSchedulerSuite extends FunSuite with LocalSparkContext with BeforeAndAfterEach {
+
+ override def afterEach() {
+ super.afterEach()
+ System.clearProperty("spark.scheduler.mode")
+ }
def createThread(threadIndex: Int, poolName: String, sc: SparkContext, sem: Semaphore) {
@@ -73,7 +76,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
TaskThreadInfo.threadToStarted(threadIndex) = new CountDownLatch(1)
new Thread {
if (poolName != null) {
- sc.setLocalProperty("spark.scheduler.cluster.fair.pool", poolName)
+ sc.setLocalProperty("spark.scheduler.pool", poolName)
}
override def run() {
val ans = nums.map(number => {
@@ -90,7 +93,7 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
}
test("Local FIFO scheduler end-to-end test") {
- System.setProperty("spark.cluster.schedulingmode", "FIFO")
+ System.setProperty("spark.scheduler.mode", "FIFO")
sc = new SparkContext("local[4]", "test")
val sem = new Semaphore(0)
@@ -148,11 +151,12 @@ class LocalSchedulerSuite extends FunSuite with LocalSparkContext {
}
test("Local fair scheduler end-to-end test") {
+ System.setProperty("spark.scheduler.mode", "FAIR")
+ val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
+ System.setProperty("spark.scheduler.allocation.file", xmlPath)
+
sc = new SparkContext("local[8]", "LocalSchedulerSuite")
val sem = new Semaphore(0)
- System.setProperty("spark.cluster.schedulingmode", "FAIR")
- val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
- System.setProperty("spark.fairscheduler.allocation.file", xmlPath)
createThread(10,"1",sc,sem)
TaskThreadInfo.threadToStarted(10).await()
diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
index 0164dda0ba..c016c51171 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -103,6 +103,27 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three")))
}
+ test("ranges") {
+ val ser = (new KryoSerializer).newInstance()
+ def check[T](t: T) {
+ assert(ser.deserialize[T](ser.serialize(t)) === t)
+ // Check that very long ranges don't get written one element at a time
+ assert(ser.serialize(t).limit < 100)
+ }
+ check(1 to 1000000)
+ check(1 to 1000000 by 2)
+ check(1 until 1000000)
+ check(1 until 1000000 by 2)
+ check(1L to 1000000L)
+ check(1L to 1000000L by 2L)
+ check(1L until 1000000L)
+ check(1L until 1000000L by 2L)
+ check(1.0 to 1000000.0 by 1.0)
+ check(1.0 to 1000000.0 by 2.0)
+ check(1.0 until 1000000.0 by 1.0)
+ check(1.0 until 1000000.0 by 2.0)
+ }
+
test("custom registrator") {
System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
new file mode 100644
index 0000000000..cb76275e39
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
@@ -0,0 +1,114 @@
+/*
+ * 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 org.scalatest.FunSuite
+
+class BlockIdSuite extends FunSuite {
+ def assertSame(id1: BlockId, id2: BlockId) {
+ assert(id1.name === id2.name)
+ assert(id1.hashCode === id2.hashCode)
+ assert(id1 === id2)
+ }
+
+ def assertDifferent(id1: BlockId, id2: BlockId) {
+ assert(id1.name != id2.name)
+ assert(id1.hashCode != id2.hashCode)
+ assert(id1 != id2)
+ }
+
+ test("test-bad-deserialization") {
+ try {
+ // Try to deserialize an invalid block id.
+ BlockId("myblock")
+ fail()
+ } catch {
+ case e: IllegalStateException => // OK
+ case _ => fail()
+ }
+ }
+
+ test("rdd") {
+ val id = RDDBlockId(1, 2)
+ assertSame(id, RDDBlockId(1, 2))
+ assertDifferent(id, RDDBlockId(1, 1))
+ assert(id.name === "rdd_1_2")
+ assert(id.asRDDId.get.rddId === 1)
+ assert(id.asRDDId.get.splitIndex === 2)
+ assert(id.isRDD)
+ assertSame(id, BlockId(id.toString))
+ }
+
+ test("shuffle") {
+ val id = ShuffleBlockId(1, 2, 3)
+ assertSame(id, ShuffleBlockId(1, 2, 3))
+ assertDifferent(id, ShuffleBlockId(3, 2, 3))
+ assert(id.name === "shuffle_1_2_3")
+ assert(id.asRDDId === None)
+ assert(id.shuffleId === 1)
+ assert(id.mapId === 2)
+ assert(id.reduceId === 3)
+ assert(id.isShuffle)
+ assertSame(id, BlockId(id.toString))
+ }
+
+ test("broadcast") {
+ val id = BroadcastBlockId(42)
+ assertSame(id, BroadcastBlockId(42))
+ assertDifferent(id, BroadcastBlockId(123))
+ assert(id.name === "broadcast_42")
+ assert(id.asRDDId === None)
+ assert(id.broadcastId === 42)
+ assert(id.isBroadcast)
+ assertSame(id, BlockId(id.toString))
+ }
+
+ test("taskresult") {
+ val id = TaskResultBlockId(60)
+ assertSame(id, TaskResultBlockId(60))
+ assertDifferent(id, TaskResultBlockId(61))
+ assert(id.name === "taskresult_60")
+ assert(id.asRDDId === None)
+ assert(id.taskId === 60)
+ assert(!id.isRDD)
+ assertSame(id, BlockId(id.toString))
+ }
+
+ test("stream") {
+ val id = StreamBlockId(1, 100)
+ assertSame(id, StreamBlockId(1, 100))
+ assertDifferent(id, StreamBlockId(2, 101))
+ assert(id.name === "input-1-100")
+ assert(id.asRDDId === None)
+ assert(id.streamId === 1)
+ assert(id.uniqueId === 100)
+ assert(!id.isBroadcast)
+ assertSame(id, BlockId(id.toString))
+ }
+
+ test("test") {
+ val id = TestBlockId("abc")
+ assertSame(id, TestBlockId("abc"))
+ assertDifferent(id, TestBlockId("ab"))
+ assert(id.name === "test_abc")
+ assert(id.asRDDId === None)
+ assert(id.id === "abc")
+ assert(!id.isShuffle)
+ assertSame(id, BlockId(id.toString))
+ }
+}
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..484a654108 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -32,7 +32,6 @@ import org.scalatest.time.SpanSugar._
import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream}
import org.apache.spark.serializer.{JavaSerializer, KryoSerializer}
-
class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester {
var store: BlockManager = null
var store2: BlockManager = null
@@ -46,6 +45,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
System.setProperty("spark.kryoserializer.buffer.mb", "1")
val serializer = new KryoSerializer
+ // Implicitly convert strings to BlockIds for test clarity.
+ implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value)
+ def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId)
+
before {
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0)
this.actorSystem = actorSystem
@@ -229,31 +232,31 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
// Putting a1, a2 and a3 in memory.
- store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY)
- store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY)
store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY)
master.removeRdd(0, blocking = false)
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
- store.getSingle("rdd_0_0") should be (None)
- master.getLocations("rdd_0_0") should have size 0
+ store.getSingle(rdd(0, 0)) should be (None)
+ master.getLocations(rdd(0, 0)) should have size 0
}
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
- store.getSingle("rdd_0_1") should be (None)
- master.getLocations("rdd_0_1") should have size 0
+ store.getSingle(rdd(0, 1)) should be (None)
+ master.getLocations(rdd(0, 1)) should have size 0
}
eventually(timeout(1000 milliseconds), interval(10 milliseconds)) {
store.getSingle("nonrddblock") should not be (None)
master.getLocations("nonrddblock") should have size (1)
}
- store.putSingle("rdd_0_0", a1, StorageLevel.MEMORY_ONLY)
- store.putSingle("rdd_0_1", a2, StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY)
master.removeRdd(0, blocking = true)
- store.getSingle("rdd_0_0") should be (None)
- master.getLocations("rdd_0_0") should have size 0
- store.getSingle("rdd_0_1") should be (None)
- master.getLocations("rdd_0_1") should have size 0
+ store.getSingle(rdd(0, 0)) should be (None)
+ master.getLocations(rdd(0, 0)) should have size 0
+ store.getSingle(rdd(0, 1)) should be (None)
+ master.getLocations(rdd(0, 1)) should have size 0
}
test("reregistration on heart beat") {
@@ -372,41 +375,41 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
- store.putSingle("rdd_0_1", a1, StorageLevel.MEMORY_ONLY)
- store.putSingle("rdd_0_2", a2, StorageLevel.MEMORY_ONLY)
- store.putSingle("rdd_0_3", a3, StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 1), a1, StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 2), a2, StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY)
// Even though we accessed rdd_0_3 last, it should not have replaced partitions 1 and 2
// from the same RDD
- assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
- assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store")
- assert(store.getSingle("rdd_0_1") != None, "rdd_0_1 was not in store")
+ assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
+ assert(store.getSingle(rdd(0, 2)) != None, "rdd_0_2 was not in store")
+ assert(store.getSingle(rdd(0, 1)) != None, "rdd_0_1 was not in store")
// Check that rdd_0_3 doesn't replace them even after further accesses
- assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
- assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
- assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
+ assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
+ assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
+ assert(store.getSingle(rdd(0, 3)) === None, "rdd_0_3 was in store")
}
test("in-memory LRU for partitions of multiple RDDs") {
store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
- store.putSingle("rdd_0_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
- store.putSingle("rdd_0_2", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
- store.putSingle("rdd_1_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
// At this point rdd_1_1 should've replaced rdd_0_1
- assert(store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was not in store")
- assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store")
- assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store")
+ assert(store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was not in store")
+ assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store")
+ assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store")
// Do a get() on rdd_0_2 so that it is the most recently used item
- assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store")
+ assert(store.getSingle(rdd(0, 2)) != None, "rdd_0_2 was not in store")
// Put in more partitions from RDD 0; they should replace rdd_1_1
- store.putSingle("rdd_0_3", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
- store.putSingle("rdd_0_4", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 3), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle(rdd(0, 4), new Array[Byte](400), StorageLevel.MEMORY_ONLY)
// Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped
// when we try to add rdd_0_4.
- assert(!store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was in store")
- assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store")
- assert(!store.memoryStore.contains("rdd_0_4"), "rdd_0_4 was in store")
- assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store")
- assert(store.memoryStore.contains("rdd_0_3"), "rdd_0_3 was not in store")
+ assert(!store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was in store")
+ assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store")
+ assert(!store.memoryStore.contains(rdd(0, 4)), "rdd_0_4 was in store")
+ assert(store.memoryStore.contains(rdd(0, 2)), "rdd_0_2 was not in store")
+ assert(store.memoryStore.contains(rdd(0, 3)), "rdd_0_3 was not in store")
}
test("on-disk storage") {
@@ -590,43 +593,46 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
try {
System.setProperty("spark.shuffle.compress", "true")
store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
- store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
- assert(store.memoryStore.getSize("shuffle_0_0_0") <= 100, "shuffle_0_0_0 was not compressed")
+ store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100,
+ "shuffle_0_0_0 was not compressed")
store.stop()
store = null
System.setProperty("spark.shuffle.compress", "false")
store = new BlockManager("exec2", actorSystem, master, serializer, 2000)
- store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
- assert(store.memoryStore.getSize("shuffle_0_0_0") >= 1000, "shuffle_0_0_0 was compressed")
+ store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000,
+ "shuffle_0_0_0 was compressed")
store.stop()
store = null
System.setProperty("spark.broadcast.compress", "true")
store = new BlockManager("exec3", actorSystem, master, serializer, 2000)
- store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
- assert(store.memoryStore.getSize("broadcast_0") <= 100, "broadcast_0 was not compressed")
+ store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100,
+ "broadcast_0 was not compressed")
store.stop()
store = null
System.setProperty("spark.broadcast.compress", "false")
store = new BlockManager("exec4", actorSystem, master, serializer, 2000)
- store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
- assert(store.memoryStore.getSize("broadcast_0") >= 1000, "broadcast_0 was compressed")
+ store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed")
store.stop()
store = null
System.setProperty("spark.rdd.compress", "true")
store = new BlockManager("exec5", actorSystem, master, serializer, 2000)
- store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
- assert(store.memoryStore.getSize("rdd_0_0") <= 100, "rdd_0_0 was not compressed")
+ store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed")
store.stop()
store = null
System.setProperty("spark.rdd.compress", "false")
store = new BlockManager("exec6", actorSystem, master, serializer, 2000)
- store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
- assert(store.memoryStore.getSize("rdd_0_0") >= 1000, "rdd_0_0 was compressed")
+ store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed")
store.stop()
store = null
diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
index 3321fb5eb7..8f0ec6683b 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
@@ -24,9 +24,14 @@ import org.eclipse.jetty.server.Server
class UISuite extends FunSuite {
test("jetty port increases under contention") {
- val startPort = 3030
+ val startPort = 4040
val server = new Server(startPort)
- server.start()
+
+ Try { server.start() } match {
+ case Success(s) =>
+ case Failure(e) =>
+ // Either case server port is busy hence setup for test complete
+ }
val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq())
val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq())
diff --git a/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala
new file mode 100644
index 0000000000..7177919a58
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/AppendOnlyMapSuite.scala
@@ -0,0 +1,154 @@
+/*
+ * 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
+
+import scala.collection.mutable.HashSet
+
+import org.scalatest.FunSuite
+
+class AppendOnlyMapSuite extends FunSuite {
+ test("initialization") {
+ val goodMap1 = new AppendOnlyMap[Int, Int](1)
+ assert(goodMap1.size === 0)
+ val goodMap2 = new AppendOnlyMap[Int, Int](255)
+ assert(goodMap2.size === 0)
+ val goodMap3 = new AppendOnlyMap[Int, Int](256)
+ assert(goodMap3.size === 0)
+ intercept[IllegalArgumentException] {
+ new AppendOnlyMap[Int, Int](1 << 30) // Invalid map size: bigger than 2^29
+ }
+ intercept[IllegalArgumentException] {
+ new AppendOnlyMap[Int, Int](-1)
+ }
+ intercept[IllegalArgumentException] {
+ new AppendOnlyMap[Int, Int](0)
+ }
+ }
+
+ test("object keys and values") {
+ val map = new AppendOnlyMap[String, String]()
+ for (i <- 1 to 100) {
+ map("" + i) = "" + i
+ }
+ assert(map.size === 100)
+ for (i <- 1 to 100) {
+ assert(map("" + i) === "" + i)
+ }
+ assert(map("0") === null)
+ assert(map("101") === null)
+ assert(map(null) === null)
+ val set = new HashSet[(String, String)]
+ for ((k, v) <- map) { // Test the foreach method
+ set += ((k, v))
+ }
+ assert(set === (1 to 100).map(_.toString).map(x => (x, x)).toSet)
+ }
+
+ test("primitive keys and values") {
+ val map = new AppendOnlyMap[Int, Int]()
+ for (i <- 1 to 100) {
+ map(i) = i
+ }
+ assert(map.size === 100)
+ for (i <- 1 to 100) {
+ assert(map(i) === i)
+ }
+ assert(map(0) === null)
+ assert(map(101) === null)
+ val set = new HashSet[(Int, Int)]
+ for ((k, v) <- map) { // Test the foreach method
+ set += ((k, v))
+ }
+ assert(set === (1 to 100).map(x => (x, x)).toSet)
+ }
+
+ test("null keys") {
+ val map = new AppendOnlyMap[String, String]()
+ for (i <- 1 to 100) {
+ map("" + i) = "" + i
+ }
+ assert(map.size === 100)
+ assert(map(null) === null)
+ map(null) = "hello"
+ assert(map.size === 101)
+ assert(map(null) === "hello")
+ }
+
+ test("null values") {
+ val map = new AppendOnlyMap[String, String]()
+ for (i <- 1 to 100) {
+ map("" + i) = null
+ }
+ assert(map.size === 100)
+ assert(map("1") === null)
+ assert(map(null) === null)
+ assert(map.size === 100)
+ map(null) = null
+ assert(map.size === 101)
+ assert(map(null) === null)
+ }
+
+ test("changeValue") {
+ val map = new AppendOnlyMap[String, String]()
+ for (i <- 1 to 100) {
+ map("" + i) = "" + i
+ }
+ assert(map.size === 100)
+ for (i <- 1 to 100) {
+ val res = map.changeValue("" + i, (hadValue, oldValue) => {
+ assert(hadValue === true)
+ assert(oldValue === "" + i)
+ oldValue + "!"
+ })
+ assert(res === i + "!")
+ }
+ // Iterate from 101 to 400 to make sure the map grows a couple of times, because we had a
+ // bug where changeValue would return the wrong result when the map grew on that insert
+ for (i <- 101 to 400) {
+ val res = map.changeValue("" + i, (hadValue, oldValue) => {
+ assert(hadValue === false)
+ i + "!"
+ })
+ assert(res === i + "!")
+ }
+ assert(map.size === 400)
+ assert(map(null) === null)
+ map.changeValue(null, (hadValue, oldValue) => {
+ assert(hadValue === false)
+ "null!"
+ })
+ assert(map.size === 401)
+ map.changeValue(null, (hadValue, oldValue) => {
+ assert(hadValue === true)
+ assert(oldValue === "null!")
+ "null!!"
+ })
+ assert(map.size === 401)
+ }
+
+ test("inserting in capacity-1 map") {
+ val map = new AppendOnlyMap[String, String](1)
+ for (i <- 1 to 100) {
+ map("" + i) = "" + i
+ }
+ assert(map.size === 100)
+ for (i <- 1 to 100) {
+ assert(map("" + i) === "" + i)
+ }
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index e2859caf58..4684c8c972 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.util
import com.google.common.base.Charsets
import com.google.common.io.Files
import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream, File}
+import java.nio.{ByteBuffer, ByteOrder}
import org.scalatest.FunSuite
import org.apache.commons.io.FileUtils
import scala.util.Random
@@ -135,5 +136,15 @@ class UtilsSuite extends FunSuite {
FileUtils.deleteDirectory(tmpDir2)
}
+
+ test("deserialize long value") {
+ val testval : Long = 9730889947L
+ val bbuf = ByteBuffer.allocate(8)
+ assert(bbuf.hasArray)
+ bbuf.order(ByteOrder.BIG_ENDIAN)
+ bbuf.putLong(testval)
+ assert(bbuf.array.length === 8)
+ assert(Utils.deserializeLongValue(bbuf.array) === testval)
+ }
}
diff --git a/docker/README.md b/docker/README.md
new file mode 100644
index 0000000000..bf59e77d11
--- /dev/null
+++ b/docker/README.md
@@ -0,0 +1,5 @@
+Spark docker files
+===========
+
+Drawn from Matt Massie's docker files (https://github.com/massie/dockerfiles),
+as well as some updates from Andre Schumacher (https://github.com/AndreSchumacher/docker). \ No newline at end of file
diff --git a/docker/build b/docker/build
new file mode 100755
index 0000000000..253a2fc8dd
--- /dev/null
+++ b/docker/build
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+docker images > /dev/null || { echo Please install docker in non-sudo mode. ; exit; }
+
+./spark-test/build \ No newline at end of file
diff --git a/docker/spark-test/README.md b/docker/spark-test/README.md
new file mode 100644
index 0000000000..addea277aa
--- /dev/null
+++ b/docker/spark-test/README.md
@@ -0,0 +1,10 @@
+Spark Docker files usable for testing and development purposes.
+
+These images are intended to be run like so:
+docker run -v $SPARK_HOME:/opt/spark spark-test-master
+docker run -v $SPARK_HOME:/opt/spark spark-test-worker <master_ip>
+
+Using this configuration, the containers will have their Spark directories
+mounted to your actual SPARK_HOME, allowing you to modify and recompile
+your Spark source and have them immediately usable in the docker images
+(without rebuilding them).
diff --git a/docker/spark-test/base/Dockerfile b/docker/spark-test/base/Dockerfile
new file mode 100644
index 0000000000..60962776dd
--- /dev/null
+++ b/docker/spark-test/base/Dockerfile
@@ -0,0 +1,38 @@
+#
+# 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.
+#
+
+FROM ubuntu:precise
+
+RUN echo "deb http://archive.ubuntu.com/ubuntu precise main universe" > /etc/apt/sources.list
+
+# Upgrade package index
+RUN apt-get update
+
+# install a few other useful packages plus Open Jdk 7
+RUN apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server
+
+ENV SCALA_VERSION 2.9.3
+ENV SPARK_VERSION 0.8.1
+ENV CDH_VERSION cdh4
+ENV SCALA_HOME /opt/scala-$SCALA_VERSION
+ENV SPARK_HOME /opt/spark
+ENV PATH $SPARK_HOME:$SCALA_HOME/bin:$PATH
+
+# Install Scala
+ADD http://www.scala-lang.org/files/archive/scala-$SCALA_VERSION.tgz /
+RUN (cd / && gunzip < scala-$SCALA_VERSION.tgz)|(cd /opt && tar -xvf -)
+RUN rm /scala-$SCALA_VERSION.tgz
diff --git a/docker/spark-test/build b/docker/spark-test/build
new file mode 100755
index 0000000000..6f9e197433
--- /dev/null
+++ b/docker/spark-test/build
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+docker build -t spark-test-base spark-test/base/
+docker build -t spark-test-master spark-test/master/
+docker build -t spark-test-worker spark-test/worker/
diff --git a/docker/spark-test/master/Dockerfile b/docker/spark-test/master/Dockerfile
new file mode 100644
index 0000000000..f729534ab6
--- /dev/null
+++ b/docker/spark-test/master/Dockerfile
@@ -0,0 +1,21 @@
+# Spark Master
+#
+# 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.
+#
+
+FROM spark-test-base
+ADD default_cmd /root/
+CMD ["/root/default_cmd"]
diff --git a/docker/spark-test/master/default_cmd b/docker/spark-test/master/default_cmd
new file mode 100755
index 0000000000..a5b1303c2e
--- /dev/null
+++ b/docker/spark-test/master/default_cmd
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }')
+echo "CONTAINER_IP=$IP"
+/opt/spark/spark-class org.apache.spark.deploy.master.Master -i $IP
diff --git a/docker/spark-test/worker/Dockerfile b/docker/spark-test/worker/Dockerfile
new file mode 100644
index 0000000000..890febe7b6
--- /dev/null
+++ b/docker/spark-test/worker/Dockerfile
@@ -0,0 +1,22 @@
+# Spark Worker
+#
+# 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.
+#
+
+FROM spark-test-base
+ENV SPARK_WORKER_PORT 8888
+ADD default_cmd /root/
+ENTRYPOINT ["/root/default_cmd"]
diff --git a/docker/spark-test/worker/default_cmd b/docker/spark-test/worker/default_cmd
new file mode 100755
index 0000000000..ab6336f70c
--- /dev/null
+++ b/docker/spark-test/worker/default_cmd
@@ -0,0 +1,22 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }')
+echo "CONTAINER_IP=$IP"
+/opt/spark/spark-class org.apache.spark.deploy.worker.Worker $1
diff --git a/docs/_config.yml b/docs/_config.yml
index 5c135a0242..48ecb8d0c9 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -3,8 +3,8 @@ markdown: kramdown
# These allow the documentation to be updated with nerw releases
# of Spark, Scala, and Mesos.
-SPARK_VERSION: 0.8.0-SNAPSHOT
-SPARK_VERSION_SHORT: 0.8.0
+SPARK_VERSION: 0.9.0-incubating-SNAPSHOT
+SPARK_VERSION_SHORT: 0.9.0-SNAPSHOT
SCALA_VERSION: 2.9.3
-MESOS_VERSION: 0.9.0-incubating
+MESOS_VERSION: 0.13.0
SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 84749fda4e..0c1d657cde 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -6,7 +6,7 @@
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1">
- <title>{{ page.title }} - Spark {{site.SPARK_VERSION}} Documentation</title>
+ <title>{{ page.title }} - Spark {{site.SPARK_VERSION_SHORT}} Documentation</title>
<meta name="description" content="">
<link rel="stylesheet" href="css/bootstrap.min.css">
@@ -51,7 +51,7 @@
<div class="navbar-inner">
<div class="container">
<div class="brand"><a href="index.html">
- <img src="img/spark-logo-77x50px-hd.png" /></a><span class="version">{{site.SPARK_VERSION_SHORT}}</span>
+ <img src="img/spark-logo-hd.png" style="height:50px;"/></a><span class="version">{{site.SPARK_VERSION_SHORT}}</span>
</div>
<ul class="nav">
<!--TODO(andyk): Add class="active" attribute to li some how.-->
@@ -86,6 +86,7 @@
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">Deploying<b class="caret"></b></a>
<ul class="dropdown-menu">
+ <li><a href="cluster-overview.html">Overview</a></li>
<li><a href="ec2-scripts.html">Amazon EC2</a></li>
<li><a href="spark-standalone.html">Standalone Mode</a></li>
<li><a href="running-on-mesos.html">Mesos</a></li>
@@ -97,14 +98,18 @@
<a href="api.html" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a>
<ul class="dropdown-menu">
<li><a href="configuration.html">Configuration</a></li>
+ <li><a href="monitoring.html">Monitoring</a></li>
<li><a href="tuning.html">Tuning Guide</a></li>
+ <li><a href="hadoop-third-party-distributions.html">Running with CDH/HDP</a></li>
<li><a href="hardware-provisioning.html">Hardware Provisioning</a></li>
+ <li><a href="job-scheduling.html">Job Scheduling</a></li>
+ <li class="divider"></li>
<li><a href="building-with-maven.html">Building Spark with Maven</a></li>
- <li><a href="contributing-to-spark.html">Contributing to Spark</a></li>
+ <li><a href="https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark">Contributing to Spark</a></li>
</ul>
</li>
</ul>
- <!--<p class="navbar-text pull-right"><span class="version-text">v{{site.SPARK_VERSION}}</span></p>-->
+ <!--<p class="navbar-text pull-right"><span class="version-text">v{{site.SPARK_VERSION_SHORT}}</span></p>-->
</div>
</div>
</div>
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index 7ecb601ddd..19c01e179f 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -37,13 +37,13 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit
# Cloudera CDH 4.2.0 with MapReduce v1
$ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package
-For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should also enable the "hadoop2-yarn" profile:
+For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "hadoop2-yarn" profile and set the "yarn.version" property:
# Apache Hadoop 2.0.5-alpha
- $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -DskipTests clean package
+ $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package
# Cloudera CDH 4.2.0 with MapReduce v2
- $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package
+ $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
## Spark Tests in Maven ##
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
new file mode 100644
index 0000000000..f679cad713
--- /dev/null
+++ b/docs/cluster-overview.md
@@ -0,0 +1,117 @@
+---
+layout: global
+title: Cluster Mode Overview
+---
+
+This document gives a short overview of how Spark runs on clusters, to make it easier to understand
+the components involved.
+
+# Components
+
+Spark applications run as independent sets of processes on a cluster, coordinated by the SparkContext
+object in your main program (called the _driver program_).
+Specifically, to run on a cluster, the SparkContext can connect to several types of _cluster managers_
+(either Spark's own standalone cluster manager or Mesos/YARN), which allocate resources across
+applications. Once connected, Spark acquires *executors* on nodes in the cluster, which are
+worker processes that run computations and store data for your application.
+Next, it sends your application code (defined by JAR or Python files passed to SparkContext) to
+the executors. Finally, SparkContext sends *tasks* for the executors to run.
+
+<p style="text-align: center;">
+ <img src="img/cluster-overview.png" title="Spark cluster components" alt="Spark cluster components" />
+</p>
+
+There are several useful things to note about this architecture:
+
+1. Each application gets its own executor processes, which stay up for the duration of the whole
+ application and run tasks in multiple threads. This has the benefit of isolating applications
+ from each other, on both the scheduling side (each driver schedules its own tasks) and executor
+ side (tasks from different applications run in different JVMs). However, it also means that
+ data cannot be shared across different Spark applications (instances of SparkContext) without
+ writing it to an external storage system.
+2. Spark is agnostic to the underlying cluster manager. As long as it can acquire executor
+ processes, and these communicate with each other, it is relatively easy to run it even on a
+ cluster manager that also supports other applications (e.g. Mesos/YARN).
+3. Because the driver schedules tasks on the cluster, it should be run close to the worker
+ nodes, preferably on the same local area network. If you'd like to send requests to the
+ cluster remotely, it's better to open an RPC to the driver and have it submit operations
+ from nearby than to run a driver far away from the worker nodes.
+
+# Cluster Manager Types
+
+The system currently supports three cluster managers:
+
+* [Standalone](spark-standalone.html) -- a simple cluster manager included with Spark that makes it
+ easy to set up a cluster.
+* [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce
+ and service applications.
+* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.0.
+
+In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone
+cluster on Amazon EC2.
+
+# Shipping Code to the Cluster
+
+The recommended way to ship your code to the cluster is to pass it through SparkContext's constructor,
+which takes a list of JAR files (Java/Scala) or .egg and .zip libraries (Python) to disseminate to
+worker nodes. You can also dynamically add new files to be sent to executors with `SparkContext.addJar`
+and `addFile`.
+
+# Monitoring
+
+Each driver program has a web UI, typically on port 4040, that displays information about running
+tasks, executors, and storage usage. Simply go to `http://<driver-node>:4040` in a web browser to
+access this UI. The [monitoring guide](monitoring.html) also describes other monitoring options.
+
+# Job Scheduling
+
+Spark gives control over resource allocation both _across_ applications (at the level of the cluster
+manager) and _within_ applications (if multiple computations are happening on the same SparkContext).
+The [job scheduling overview](job-scheduling.html) describes this in more detail.
+
+# Glossary
+
+The following table summarizes terms you'll see used to refer to cluster concepts:
+
+<table class="table">
+ <thead>
+ <tr><th style="width: 130px;">Term</th><th>Meaning</th></tr>
+ </thead>
+ <tbody>
+ <tr>
+ <td>Application</td>
+ <td>User program built on Spark. Consists of a <em>driver program</em> and <em>executors</em> on the cluster.</td>
+ </tr>
+ <tr>
+ <td>Driver program</td>
+ <td>The process running the main() function of the application and creating the SparkContext</td>
+ </tr>
+ <tr>
+ <td>Cluster manager</td>
+ <td>An external service for acquiring resources on the cluster (e.g. standalone manager, Mesos, YARN)</td>
+ </tr>
+ <tr>
+ <td>Worker node</td>
+ <td>Any node that can run application code in the cluster</td>
+ </tr>
+ <tr>
+ <td>Executor</td>
+ <td>A process launched for an application on a worker node, that runs tasks and keeps data in memory
+ or disk storage across them. Each application has its own executors.</td>
+ </tr>
+ <tr>
+ <td>Task</td>
+ <td>A unit of work that will be sent to one executor</td>
+ </tr>
+ <tr>
+ <td>Job</td>
+ <td>A parallel computation consisting of multiple tasks that gets spawned in response to a Spark action
+ (e.g. <code>save</code>, <code>collect</code>); you'll see this term used in the driver's logs.</td>
+ </tr>
+ <tr>
+ <td>Stage</td>
+ <td>Each job gets divided into smaller sets of tasks called <em>stages</em> that depend on each other
+ (similar to the map and reduce stages in MapReduce); you'll see this term used in the driver's logs.</td>
+ </tr>
+ </tbody>
+</table>
diff --git a/docs/configuration.md b/docs/configuration.md
index 310e78a9eb..7940d41a27 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -82,17 +82,6 @@ Apart from these, the following properties are also available, and may be useful
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
- <td>spark.mesos.coarse</td>
- <td>false</td>
- <td>
- If set to "true", runs over Mesos clusters in
- <a href="running-on-mesos.html#mesos-run-modes">"coarse-grained" sharing mode</a>,
- where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task.
- This gives lower-latency scheduling for short queries, but leaves resources in use for the whole
- duration of the Spark job.
- </td>
-</tr>
-<tr>
<td>spark.default.parallelism</td>
<td>8</td>
<td>
@@ -110,8 +99,19 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
+ <td>spark.mesos.coarse</td>
+ <td>false</td>
+ <td>
+ If set to "true", runs over Mesos clusters in
+ <a href="running-on-mesos.html#mesos-run-modes">"coarse-grained" sharing mode</a>,
+ where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task.
+ This gives lower-latency scheduling for short queries, but leaves resources in use for the whole
+ duration of the Spark job.
+ </td>
+</tr>
+<tr>
<td>spark.ui.port</td>
- <td>3030</td>
+ <td>4040</td>
<td>
Port for your application's dashboard, which shows memory and workload data
</td>
@@ -161,6 +161,16 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
+ <td>spark.scheduler.mode</td>
+ <td>FIFO</td>
+ <td>
+ The <a href="job-scheduling.html#scheduling-within-an-application">scheduling mode</a> between
+ jobs submitted to the same SparkContext. Can be set to <code>FAIR</code>
+ to use fair sharing instead of queueing jobs one after another. Useful for
+ multi-user services.
+ </td>
+</tr>
+<tr>
<td>spark.reducer.maxMbInFlight</td>
<td>48</td>
<td>
diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md
index 50feeb2d6c..ef1b3ad6da 100644
--- a/docs/contributing-to-spark.md
+++ b/docs/contributing-to-spark.md
@@ -3,24 +3,6 @@ layout: global
title: Contributing to Spark
---
-The Spark team welcomes contributions in the form of GitHub pull requests. Here are a few tips to get your contribution in:
-
-- Break your work into small, single-purpose patches if possible. It's much harder to merge in a large change with a lot of disjoint features.
-- Submit the patch as a GitHub pull request. For a tutorial, see the GitHub guides on [forking a repo](https://help.github.com/articles/fork-a-repo) and [sending a pull request](https://help.github.com/articles/using-pull-requests).
-- Follow the style of the existing codebase. Specifically, we use [standard Scala style guide](http://docs.scala-lang.org/style/), but with the following changes:
- * Maximum line length of 100 characters.
- * Always import packages using absolute paths (e.g. `scala.collection.Map` instead of `collection.Map`).
- * No "infix" syntax for methods other than operators. For example, don't write `table containsKey myKey`; replace it with `table.containsKey(myKey)`.
-- Make sure that your code passes the unit tests. You can run the tests with `sbt/sbt test` in the root directory of Spark.
- But first, make sure that you have [configured a spark-env.sh](configuration.html) with at least
- `SCALA_HOME`, as some of the tests try to spawn subprocesses using this.
-- Add new unit tests for your code. We use [ScalaTest](http://www.scalatest.org/) for testing. Just add a new Suite in `core/src/test`, or methods to an existing Suite.
-- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issue tracker]({{site.SPARK_ISSUE_TRACKER_URL}}), or email the [mailing list](http://www.spark-project.org/mailing-lists.html).
-
-# Licensing of Contributions
-
-Contributions via GitHub pull requests are gladly accepted from their original author. Along with any pull requests, please
-state that the contribution is your original work and that you license the work to the project under the project's open source
-license. *Whether or not you state this explicitly, by submitting any copyrighted material via pull request, email, or other
-means you agree to license the material under the project's open source license and warrant that you have the legal authority
-to do so.*
+The Spark team welcomes all forms of contributions, including bug reports, documentation or patches.
+For the newest information on how to contribute to the project, please read the
+[wiki page on contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark).
diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md
index da0c06e2a6..1e5575d657 100644
--- a/docs/ec2-scripts.md
+++ b/docs/ec2-scripts.md
@@ -80,7 +80,7 @@ another.
permissions on your private key file, you can run `launch` with the
`--resume` option to restart the setup process on an existing cluster.
-# Running Jobs
+# Running Applications
- Go into the `ec2` directory in the release of Spark you downloaded.
- Run `./spark-ec2 -k <keypair> -i <key-file> login <cluster-name>` to
@@ -90,7 +90,7 @@ permissions on your private key file, you can run `launch` with the
- To deploy code or data within your cluster, you can log in and use the
provided script `~/spark-ec2/copy-dir`, which,
given a directory path, RSYNCs it to the same location on all the slaves.
-- If your job needs to access large datasets, the fastest way to do
+- If your application needs to access large datasets, the fastest way to do
that is to load them from Amazon S3 or an Amazon EBS device into an
instance of the Hadoop Distributed File System (HDFS) on your nodes.
The `spark-ec2` script already sets up a HDFS instance for you. It's
@@ -103,8 +103,8 @@ permissions on your private key file, you can run `launch` with the
(about 3 GB), but you can use the `--ebs-vol-size` option to
`spark-ec2` to attach a persistent EBS volume to each node for
storing the persistent HDFS.
-- Finally, if you get errors while running your jobs, look at the slave's logs
- for that job inside of the scheduler work directory (/root/spark/work). You can
+- Finally, if you get errors while running your application, look at the slave's logs
+ for that application inside of the scheduler work directory (/root/spark/work). You can
also view the status of the cluster using the web UI: `http://<master-hostname>:8080`.
# Configuration
diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md
new file mode 100644
index 0000000000..f706625fe9
--- /dev/null
+++ b/docs/hadoop-third-party-distributions.md
@@ -0,0 +1,118 @@
+---
+layout: global
+title: Running with Cloudera and HortonWorks
+---
+
+Spark can run against all versions of Cloudera's Distribution Including Apache Hadoop (CDH) and
+the Hortonworks Data Platform (HDP). There are a few things to keep in mind when using Spark
+with these distributions:
+
+# Compile-time Hadoop Version
+
+When compiling Spark, you'll need to
+[set the SPARK_HADOOP_VERSION flag](http://localhost:4000/index.html#a-note-about-hadoop-versions):
+
+ SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly
+
+The table below lists the corresponding `SPARK_HADOOP_VERSION` code for each CDH/HDP release. Note that
+some Hadoop releases are binary compatible across client versions. This means the pre-built Spark
+distribution may "just work" without you needing to compile. That said, we recommend compiling with
+the _exact_ Hadoop version you are running to avoid any compatibility errors.
+
+<table>
+ <tr valign="top">
+ <td>
+ <h3>CDH Releases</h3>
+ <table class="table" style="width:350px; margin-right: 20px;">
+ <tr><th>Release</th><th>Version code</th></tr>
+ <tr><td>CDH 4.X.X (YARN mode)</td><td>2.0.0-chd4.X.X</td></tr>
+ <tr><td>CDH 4.X.X</td><td>2.0.0-mr1-chd4.X.X</td></tr>
+ <tr><td>CDH 3u6</td><td>0.20.2-cdh3u6</td></tr>
+ <tr><td>CDH 3u5</td><td>0.20.2-cdh3u5</td></tr>
+ <tr><td>CDH 3u4</td><td>0.20.2-cdh3u4</td></tr>
+ </table>
+ </td>
+ <td>
+ <h3>HDP Releases</h3>
+ <table class="table" style="width:350px;">
+ <tr><th>Release</th><th>Version code</th></tr>
+ <tr><td>HDP 1.3</td><td>1.2.0</td></tr>
+ <tr><td>HDP 1.2</td><td>1.1.2</td></tr>
+ <tr><td>HDP 1.1</td><td>1.0.3</td></tr>
+ <tr><td>HDP 1.0</td><td>1.0.3</td></tr>
+ </table>
+ </td>
+ </tr>
+</table>
+
+# Linking Applications to the Hadoop Version
+
+In addition to compiling Spark itself against the right version, you need to add a Maven dependency on that
+version of `hadoop-client` to any Spark applications you run, so they can also talk to the HDFS version
+on the cluster. If you are using CDH, you also need to add the Cloudera Maven repository.
+This looks as follows in SBT:
+
+{% highlight scala %}
+libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "<version>"
+
+// If using CDH, also add Cloudera repo
+resolvers += "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
+{% endhighlight %}
+
+Or in Maven:
+
+{% highlight xml %}
+<project>
+ <dependencies>
+ ...
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-client</artifactId>
+ <version>[version]</version>
+ </dependency>
+ </dependencies>
+
+ <!-- If using CDH, also add Cloudera repo -->
+ <repositories>
+ ...
+ <repository>
+ <id>Cloudera repository</id>
+ <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
+ </repository>
+ </repositories>
+</project>
+
+{% endhighlight %}
+
+# Where to Run Spark
+
+As described in the [Hardware Provisioning](hardware-provisioning.html#storage-systems) guide,
+Spark can run in a variety of deployment modes:
+
+* Using dedicated set of Spark nodes in your cluster. These nodes should be co-located with your
+ Hadoop installation.
+* Running on the same nodes as an existing Hadoop installation, with a fixed amount memory and
+ cores dedicated to Spark on each node.
+* Run Spark alongside Hadoop using a cluster resource manager, such as YARN or Mesos.
+
+These options are identical for those using CDH and HDP.
+
+# Inheriting Cluster Configuration
+
+If you plan to read and write from HDFS using Spark, there are two Hadoop configuration files that
+should be included on Spark's classpath:
+
+* `hdfs-site.xml`, which provides default behaviors for the HDFS client.
+* `core-site.xml`, which sets the default filesystem name.
+
+The location of these configuration files varies across CDH and HDP versions, but
+a common location is inside of `/etc/hadoop/conf`. Some tools, such as Cloudera Manager, create
+configurations on-the-fly, but offer a mechanisms to download copies of them.
+
+There are a few ways to make these files visible to Spark:
+
+* You can copy these files into `$SPARK_HOME/conf` and they will be included in Spark's
+classpath automatically.
+* If you are running Spark on the same nodes as Hadoop _and_ your distribution includes both
+`hdfs-site.xml` and `core-site.xml` in the same directory, you can set `HADOOP_CONF_DIR`
+in `$SPARK_HOME/spark-env.sh` to that directory.
diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md
index e5f054cb14..790220500a 100644
--- a/docs/hardware-provisioning.md
+++ b/docs/hardware-provisioning.md
@@ -43,7 +43,7 @@ rest for the operating system and buffer cache.
How much memory you will need will depend on your application. To determine how much your
application uses for a certain dataset size, load part of your dataset in a Spark RDD and use the
-Storage tab of Spark's monitoring UI (`http://<driver-node>:3030`) to see its size in memory.
+Storage tab of Spark's monitoring UI (`http://<driver-node>:4040`) to see its size in memory.
Note that memory usage is greatly affected by storage level and serialization format -- see
the [tuning guide](tuning.html) for tips on how to reduce it.
@@ -59,7 +59,7 @@ In our experience, when the data is in memory, a lot of Spark applications are n
Using a **10 Gigabit** or higher network is the best way to make these applications faster.
This is especially true for "distributed reduce" applications such as group-bys, reduce-bys, and
SQL joins. In any given application, you can see how much data Spark shuffles across the network
-from the application's monitoring UI (`http://<driver-node>:3030`).
+from the application's monitoring UI (`http://<driver-node>:4040`).
# CPU Cores
diff --git a/docs/img/cluster-overview.png b/docs/img/cluster-overview.png
new file mode 100644
index 0000000000..368274068e
--- /dev/null
+++ b/docs/img/cluster-overview.png
Binary files differ
diff --git a/docs/img/cluster-overview.pptx b/docs/img/cluster-overview.pptx
new file mode 100644
index 0000000000..af3c462cd9
--- /dev/null
+++ b/docs/img/cluster-overview.pptx
Binary files differ
diff --git a/docs/img/spark-logo-hd.png b/docs/img/spark-logo-hd.png
new file mode 100644
index 0000000000..1381e3004d
--- /dev/null
+++ b/docs/img/spark-logo-hd.png
Binary files differ
diff --git a/docs/index.md b/docs/index.md
index 7d73929940..bd386a8a8f 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -21,7 +21,7 @@ Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with
For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VERSION}}. If you write applications in Scala, you will need to use this same version of Scala in your own program -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/).
-# Testing the Build
+# Running the Examples and Shell
Spark comes with several sample programs in the `examples` directory.
To run one of the samples, use `./run-example <class> <params>` in the top-level Spark directory
@@ -34,14 +34,16 @@ to connect to. This can be a [URL for a distributed cluster](scala-programming-g
or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using
`local` for testing.
-Finally, Spark can be used interactively through modified versions of the Scala shell (`./spark-shell`) or
-Python interpreter (`./pyspark`). These are a great way to learn Spark.
+Finally, you can run Spark interactively through modified versions of the Scala shell (`./spark-shell`) or
+Python interpreter (`./pyspark`). These are a great way to learn the framework.
-# Running on a Cluster
+# Launching on a Cluster
-Spark supports several options for deployment:
+The Spark [cluster mode overview](cluster-overview.html) explains the key concepts in running on a cluster.
+Spark can run both by itself, or over several existing cluster managers. It currently provides several
+options for deployment:
-* [Amazon EC2](ec2-scripts.html): our scripts let you launch a cluster in about 5 minutes
+* [Amazon EC2](ec2-scripts.html): our EC2 scripts let you launch a cluster in about 5 minutes
* [Standalone Deploy Mode](spark-standalone.html): simplest way to deploy Spark on a private cluster
* [Apache Mesos](running-on-mesos.html)
* [Hadoop YARN](running-on-yarn.html)
@@ -86,19 +88,21 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set
**Deployment guides:**
-* [Running Spark on Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes
+* [Cluster Overview](cluster-overview.html): overview of concepts and components when running on a cluster
+* [Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes
* [Standalone Deploy Mode](spark-standalone.html): launch a standalone cluster quickly without a third-party cluster manager
-* [Running Spark on Mesos](running-on-mesos.html): deploy a private cluster using
+* [Mesos](running-on-mesos.html): deploy a private cluster using
[Apache Mesos](http://incubator.apache.org/mesos)
-* [Running Spark on YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
+* [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
**Other documents:**
* [Configuration](configuration.html): customize Spark via its configuration system
* [Tuning Guide](tuning.html): best practices to optimize performance and memory use
* [Hardware Provisioning](hardware-provisioning.html): recommendations for cluster hardware
-* [Building Spark with Maven](building-with-maven.html): Build Spark using the Maven build tool
-* [Contributing to Spark](contributing-to-spark.html)
+* [Job Scheduling](job-scheduling.html): scheduling resources across and within Spark applications
+* [Building Spark with Maven](building-with-maven.html): build Spark using the Maven system
+* [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark)
**External resources:**
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
new file mode 100644
index 0000000000..d304c5497b
--- /dev/null
+++ b/docs/job-scheduling.md
@@ -0,0 +1,168 @@
+---
+layout: global
+title: Job Scheduling
+---
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+# Overview
+
+Spark has several facilities for scheduling resources between computations. First, recall that, as described
+in the [cluster mode overview](cluster-overview.html), each Spark application (instance of SparkContext)
+runs an independent set of executor processes. The cluster managers that Spark runs on provide
+facilities for [scheduling across applications](#scheduling-across-applications). Second,
+_within_ each Spark application, multiple "jobs" (Spark actions) may be running concurrently
+if they were submitted by different threads. This is common if your application is serving requests
+over the network; for example, the [Shark](http://shark.cs.berkeley.edu) server works this way. Spark
+includes a [fair scheduler](#scheduling-within-an-application) to schedule resources within each SparkContext.
+
+# Scheduling Across Applications
+
+When running on a cluster, each Spark application gets an independent set of executor JVMs that only
+run tasks and store data for that application. If multiple users need to share your cluster, there are
+different options to manage allocation, depending on the cluster manager.
+
+The simplest option, available on all cluster managers, is _static partitioning_ of resources. With
+this approach, each application is given a maximum amount of resources it can use, and holds onto them
+for its whole duration. This is the approach used in Spark's [standalone](spark-standalone.html)
+and [YARN](running-on-yarn.html) modes, as well as the
+[coarse-grained Mesos mode](running-on-mesos.html#mesos-run-modes).
+Resource allocation can be configured as follows, based on the cluster type:
+
+* **Standalone mode:** By default, applications submitted to the standalone mode cluster will run in
+ FIFO (first-in-first-out) order, and each application will try to use all available nodes. You can limit
+ the number of nodes an application uses by setting the `spark.cores.max` system property in it. This
+ will allow multiple users/applications to run concurrently. For example, you might launch a long-running
+ server that uses 10 cores, and allow users to launch shells that use 20 cores each.
+ Finally, in addition to controlling cores, each application's `spark.executor.memory` setting controls
+ its memory use.
+* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` system property to `true`,
+ and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode.
+ You should also set `spark.executor.memory` to control the executor memory.
+* **YARN:** The `--num-workers` option to the Spark YARN client controls how many workers it will allocate
+ on the cluster, while `--worker-memory` and `--worker-cores` control the resources per worker.
+
+A second option available on Mesos is _dynamic sharing_ of CPU cores. In this mode, each Spark application
+still has a fixed and independent memory allocation (set by `spark.executor.memory`), but when the
+application is not running tasks on a machine, other applications may run tasks on those cores. This mode
+is useful when you expect large numbers of not overly active applications, such as shell sessions from
+separate users. However, it comes with a risk of less predictable latency, because it may take a while for
+an application to gain back cores on one node when it has work to do. To use this mode, simply use a
+`mesos://` URL without setting `spark.mesos.coarse` to true.
+
+Note that none of the modes currently provide memory sharing across applications. If you would like to share
+data this way, we recommend running a single server application that can serve multiple requests by querying
+the same RDDs. For example, the [Shark](http://shark.cs.berkeley.edu) JDBC server works this way for SQL
+queries. In future releases, in-memory storage systems such as [Tachyon](http://tachyon-project.org) will
+provide another approach to share RDDs.
+
+
+# Scheduling Within an Application
+
+Inside a given Spark application (SparkContext instance), multiple parallel jobs can run simultaneously if
+they were submitted from separate threads. By "job", in this section, we mean a Spark action (e.g. `save`,
+`collect`) and any tasks that need to run to evaluate that action. Spark's scheduler is fully thread-safe
+and supports this use case to enable applications that serve multiple requests (e.g. queries for
+multiple users).
+
+By default, Spark's scheduler runs jobs in FIFO fashion. Each job is divided into "stages" (e.g. map and
+reduce phases), and the first job gets priority on all available resources while its stages have tasks to
+launch, then the second job gets priority, etc. If the jobs at the head of the queue don't need to use
+the whole cluster, later jobs can start to run right away, but if the jobs at the head of the queue are
+large, then later jobs may be delayed significantly.
+
+Starting in Spark 0.8, it is also possible to configure fair sharing between jobs. Under fair sharing,
+Spark assigns tasks between jobs in a "round robin" fashion, so that all jobs get a roughly equal share
+of cluster resources. This means that short jobs submitted while a long job is running can start receiving
+resources right away and still get good response times, without waiting for the long job to finish. This
+mode is best for multi-user settings.
+
+To enable the fair scheduler, simply set the `spark.scheduler.mode` to `FAIR` before creating
+a SparkContext:
+
+{% highlight scala %}
+System.setProperty("spark.scheduler.mode", "FAIR")
+{% endhighlight %}
+
+## Fair Scheduler Pools
+
+The fair scheduler also supports grouping jobs into _pools_, and setting different scheduling options
+(e.g. weight) for each pool. This can be useful to create a "high-priority" pool for more important jobs,
+for example, or to group the jobs of each user together and give _users_ equal shares regardless of how
+many concurrent jobs they have instead of giving _jobs_ equal shares. This approach is modeled after the
+[Hadoop Fair Scheduler](http://hadoop.apache.org/docs/stable/fair_scheduler.html).
+
+Without any intervention, newly submitted jobs go into a _default pool_, but jobs' pools can be set by
+adding the `spark.scheduler.pool` "local property" to the SparkContext in the thread that's submitting them.
+This is done as follows:
+
+{% highlight scala %}
+// Assuming context is your SparkContext variable
+context.setLocalProperty("spark.scheduler.pool", "pool1")
+{% endhighlight %}
+
+After setting this local property, _all_ jobs submitted within this thread (by calls in this thread
+to `RDD.save`, `count`, `collect`, etc) will use this pool name. The setting is per-thread to make
+it easy to have a thread run multiple jobs on behalf of the same user. If you'd like to clear the
+pool that a thread is associated with, simply call:
+
+{% highlight scala %}
+context.setLocalProperty("spark.scheduler.pool", null)
+{% endhighlight %}
+
+## Default Behavior of Pools
+
+By default, each pool gets an equal share of the cluster (also equal in share to each job in the default
+pool), but inside each pool, jobs run in FIFO order. For example, if you create one pool per user, this
+means that each user will get an equal share of the cluster, and that each user's queries will run in
+order instead of later queries taking resources from that user's earlier ones.
+
+## Configuring Pool Properties
+
+Specific pools' properties can also be modified through a configuration file. Each pool supports three
+properties:
+
+* `schedulingMode`: This can be FIFO or FAIR, to control whether jobs within the pool queue up behind
+ each other (the default) or share the pool's resources fairly.
+* `weight`: This controls the pool's share of the cluster relative to other pools. By default, all pools
+ have a weight of 1. If you give a specific pool a weight of 2, for example, it will get 2x more
+ resources as other active pools. Setting a high weight such as 1000 also makes it possible to implement
+ _priority_ between pools---in essence, the weight-1000 pool will always get to launch tasks first
+ whenever it has jobs active.
+* `minShare`: Apart from an overall weight, each pool can be given a _minimum shares_ (as a number of
+ CPU cores) that the administrator would like it to have. The fair scheduler always attempts to meet
+ all active pools' minimum shares before redistributing extra resources according to the weights.
+ The `minShare` property can therefore be another way to ensure that a pool can always get up to a
+ certain number of resources (e.g. 10 cores) quickly without giving it a high priority for the rest
+ of the cluster. By default, each pool's `minShare` is 0.
+
+The pool properties can be set by creating an XML file, similar to `conf/fairscheduler.xml.template`,
+and setting the `spark.scheduler.allocation.file` property:
+
+{% highlight scala %}
+System.setProperty("spark.scheduler.allocation.file", "/path/to/file")
+{% endhighlight %}
+
+The format of the XML file is simply a `<pool>` element for each pool, with different elements
+within it for the various settings. For example:
+
+{% highlight xml %}
+<?xml version="1.0"?>
+<allocations>
+ <pool name="production">
+ <schedulingMode>FAIR</schedulingMode>
+ <weight>1</weight>
+ <minShare>2</minShare>
+ </pool>
+ <pool name="test">
+ <schedulingMode>FIFO</schedulingMode>
+ <weight>2</weight>
+ <minShare>3</minShare>
+ </pool>
+</allocations>
+{% endhighlight %}
+
+A full example is also available in `conf/fairscheduler.xml.template`. Note that any pools not
+configured in the XML file will simply get default values for all settings (scheduling mode FIFO,
+weight 1, and minShare 0).
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index c897f8b36c..c1ff9c417c 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -3,4 +3,210 @@ layout: global
title: Machine Learning Library (MLlib)
---
-Coming soon.
+MLlib is a Spark implementation of some common machine learning (ML)
+functionality, as well associated tests and data generators. MLlib
+currently supports four common types of machine learning problem settings,
+namely, binary classification, regression, clustering and collaborative
+filtering, as well as an underlying gradient descent optimization primitive.
+This guide will outline the functionality supported in MLlib and also provides
+an example of invoking MLlib.
+
+# Dependencies
+MLlib uses the [jblas](https://github.com/mikiobraun/jblas) linear algebra library, which itself
+depends on native Fortran routines. You may need to install the
+[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries)
+if it is not already present on your nodes. MLlib will throw a linking error if it cannot
+detect these libraries automatically.
+
+# Binary Classification
+
+Binary classification is a supervised learning problem in which we want to
+classify entities into one of two distinct categories or labels, e.g.,
+predicting whether or not emails are spam. This problem involves executing a
+learning *Algorithm* on a set of *labeled* examples, i.e., a set of entities
+represented via (numerical) features along with underlying category labels.
+The algorithm returns a trained *Model* that can predict the label for new
+entities for which the underlying label is unknown.
+
+MLlib currently supports two standard model families for binary classification,
+namely [Linear Support Vector Machines
+(SVMs)](http://en.wikipedia.org/wiki/Support_vector_machine) and [Logistic
+Regression](http://en.wikipedia.org/wiki/Logistic_regression), along with [L1
+and L2 regularized](http://en.wikipedia.org/wiki/Regularization_(mathematics))
+variants of each model family. The training algorithms all leverage an
+underlying gradient descent primitive (described
+[below](#gradient-descent-primitive)), and take as input a regularization
+parameter (*regParam*) along with various parameters associated with gradient
+descent (*stepSize*, *numIterations*, *miniBatchFraction*).
+
+The following code snippet illustrates how to load a sample dataset, execute a
+training algorithm on this training data using a static method in the algorithm
+object, and make predictions with the resulting model to compute the training
+error.
+
+{% highlight scala %}
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.classification.SVMWithSGD
+import org.apache.spark.mllib.regression.LabeledPoint
+
+// Load and parse the data file
+val data = sc.textFile("mllib/data/sample_svm_data.txt")
+val parsedData = data.map { line =>
+ val parts = line.split(' ')
+ LabeledPoint(parts(0).toDouble, parts.tail.map(x => x.toDouble).toArray)
+}
+
+// Run training algorithm
+val numIterations = 20
+val model = SVMWithSGD.train(parsedData, numIterations)
+
+// Evaluate model on training examples and compute training error
+val labelAndPreds = parsedData.map { point =>
+ val prediction = model.predict(point.features)
+ (point.label, prediction)
+}
+val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count
+println("trainError = " + trainErr)
+{% endhighlight %}
+
+The `SVMWithSGD.train()` method by default performs L2 regularization with the
+regularization parameter set to 1.0. If we want to configure this algorithm, we
+can customize `SVMWithSGD` further by creating a new object directly and
+calling setter methods. All other MLlib algorithms support customization in
+this way as well. For example, the following code produces an L1 regularized
+variant of SVMs with regularization parameter set to 0.1, and runs the training
+algorithm for 200 iterations.
+
+{% highlight scala %}
+import org.apache.spark.mllib.optimization.L1Updater
+
+val svmAlg = new SVMWithSGD()
+svmAlg.optimizer.setNumIterations(200)
+ .setRegParam(0.1)
+ .setUpdater(new L1Updater)
+val modelL1 = svmAlg.run(parsedData)
+{% endhighlight %}
+
+Both of the code snippets above can be executed in `spark-shell` to generate a
+classifier for the provided dataset.
+
+Available algorithms for binary classification:
+
+* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD)
+* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD)
+
+# Linear Regression
+
+Linear regression is another classical supervised learning setting. In this
+problem, each entity is associated with a real-valued label (as opposed to a
+binary label as in binary classification), and we want to predict labels as
+closely as possible given numerical features representing entities. MLlib
+supports linear regression as well as L1
+([lasso](http://en.wikipedia.org/wiki/Lasso_(statistics)#Lasso_method)) and L2
+([ridge](http://en.wikipedia.org/wiki/Ridge_regression)) regularized variants.
+The regression algorithms in MLlib also leverage the underlying gradient
+descent primitive (described [below](#gradient-descent-primitive)), and have
+the same parameters as the binary classification algorithms described above.
+
+Available algorithms for linear regression:
+
+* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD)
+* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD)
+* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD)
+
+# Clustering
+
+Clustering is an unsupervised learning problem whereby we aim to group subsets
+of entities with one another based on some notion of similarity. Clustering is
+often used for exploratory analysis and/or as a component of a hierarchical
+supervised learning pipeline (in which distinct classifiers or regression
+models are trained for each cluster). MLlib supports
+[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, arguably
+the most commonly used clustering approach that clusters the data points into
+*k* clusters. The MLlib implementation includes a parallelized
+variant of the [k-means++](http://en.wikipedia.org/wiki/K-means%2B%2B) method
+called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf).
+The implementation in MLlib has the following parameters:
+
+* *k* is the number of clusters.
+* *maxIterations* is the maximum number of iterations to run.
+* *initializationMode* specifies either random initialization or
+initialization via k-means\|\|.
+* *runs* is the number of times to run the k-means algorithm (k-means is not
+guaranteed to find a globally optimal solution, and when run multiple times on
+a given dataset, the algorithm returns the best clustering result).
+* *initializiationSteps* determines the number of steps in the k-means\|\| algorithm.
+* *epsilon* determines the distance threshold within which we consider k-means to have converged.
+
+Available algorithms for clustering:
+
+* [KMeans](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans)
+
+# Collaborative Filtering
+
+[Collaborative filtering](http://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering)
+is commonly used for recommender systems. These techniques aim to fill in the
+missing entries of a user-item association matrix. MLlib currently supports
+model-based collaborative filtering, in which users and products are described
+by a small set of latent factors that can be used to predict missing entries.
+In particular, we implement the [alternating least squares
+(ALS)](http://www2.research.att.com/~volinsky/papers/ieeecomputer.pdf)
+algorithm to learn these latent factors. The implementation in MLlib has the
+following parameters:
+
+* *numBlocks* is the number of blacks used to parallelize computation (set to -1 to auto-configure).
+* *rank* is the number of latent factors in our model.
+* *iterations* is the number of iterations to run.
+* *lambda* specifies the regularization parameter in ALS.
+* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for *implicit feedback* data
+* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the *baseline* confidence in preference observations
+
+## Explicit vs Implicit Feedback
+
+The standard approach to matrix factorization based collaborative filtering treats
+the entries in the user-item matrix as *explicit* preferences given by the user to the item.
+
+It is common in many real-world use cases to only have access to *implicit feedback*
+(e.g. views, clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with
+such data is taken from
+[Collaborative Filtering for Implicit Feedback Datasets](http://research.yahoo.com/pub/2433).
+Essentially instead of trying to model the matrix of ratings directly, this approach treats the data as
+a combination of binary preferences and *confidence values*. The ratings are then related
+to the level of confidence in observed user preferences, rather than explicit ratings given to items.
+The model then tries to find latent factors that can be used to predict the expected preference of a user
+for an item.
+
+Available algorithms for collaborative filtering:
+
+* [ALS](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS)
+
+# Gradient Descent Primitive
+
+[Gradient descent](http://en.wikipedia.org/wiki/Gradient_descent) (along with
+stochastic variants thereof) are first-order optimization methods that are
+well-suited for large-scale and distributed computation. Gradient descent
+methods aim to find a local minimum of a function by iteratively taking steps
+in the direction of the negative gradient of the function at the current point,
+i.e., the current parameter value. Gradient descent is included as a low-level
+primitive in MLlib, upon which various ML algorithms are developed, and has the
+following parameters:
+
+* *gradient* is a class that computes the stochastic gradient of the function
+being optimized, i.e., with respect to a single training example, at the
+current parameter value. MLlib includes gradient classes for common loss
+functions, e.g., hinge, logistic, least-squares. The gradient class takes as
+input a training example, its label, and the current parameter value.
+* *updater* is a class that updates weights in each iteration of gradient
+descent. MLlib includes updaters for cases without regularization, as well as
+L1 and L2 regularizers.
+* *stepSize* is a scalar value denoting the initial step size for gradient
+descent. All updaters in MLlib use a step size at the t-th step equal to
+stepSize / sqrt(t).
+* *numIterations* is the number of iterations to run.
+* *regParam* is the regularization parameter when using L1 or L2 regularization.
+* *miniBatchFraction* is the fraction of the data used to compute the gradient
+at each iteration.
+
+Available algorithms for gradient descent:
+
+* [GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent)
diff --git a/docs/monitoring.md b/docs/monitoring.md
new file mode 100644
index 0000000000..5f456b999b
--- /dev/null
+++ b/docs/monitoring.md
@@ -0,0 +1,70 @@
+---
+layout: global
+title: Monitoring and Instrumentation
+---
+
+There are several ways to monitor Spark applications.
+
+# Web Interfaces
+
+Every SparkContext launches a web UI, by default on port 4040, that
+displays useful information about the application. This includes:
+
+* A list of scheduler stages and tasks
+* A summary of RDD sizes and memory usage
+* Information about the running executors
+* Environmental information.
+
+You can access this interface by simply opening `http://<driver-node>:4040` in a web browser.
+If multiple SparkContexts are running on the same host, they will bind to succesive ports
+beginning with 4040 (4041, 4042, etc).
+
+Spark's Standlone Mode cluster manager also has its own
+[web UI](spark-standalone.html#monitoring-and-logging).
+
+Note that in both of these UIs, the tables are sortable by clicking their headers,
+making it easy to identify slow tasks, data skew, etc.
+
+# Metrics
+
+Spark has a configurable metrics system based on the
+[Coda Hale Metrics Library](http://metrics.codahale.com/).
+This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV
+files. The metrics system is configured via a configuration file that Spark expects to be present
+at `$SPARK_HOME/conf/metrics.conf`. A custom file location can be specified via the
+`spark.metrics.conf` Java system property. Spark's metrics are decoupled into different
+_instances_ corresponding to Spark components. Within each instance, you can configure a
+set of sinks to which metrics are reported. The following instances are currently supported:
+
+* `master`: The Spark standalone master process.
+* `applications`: A component within the master which reports on various applications.
+* `worker`: A Spark standalone worker process.
+* `executor`: A Spark executor.
+* `driver`: The Spark driver process (the process in which your SparkContext is created).
+
+Each instance can report to zero or more _sinks_. Sinks are contained in the
+`org.apache.spark.metrics.sink` package:
+
+* `ConsoleSink`: Logs metrics information to the console.
+* `CSVSink`: Exports metrics data to CSV files at regular intervals.
+* `GangliaSink`: Sends metrics to a Ganglia node or multicast group.
+* `JmxSink`: Registers metrics for viewing in a JXM console.
+* `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data.
+
+The syntax of the metrics configuration file is defined in an example configuration file,
+`$SPARK_HOME/conf/metrics.conf.template`.
+
+# Advanced Instrumentation
+
+Several external tools can be used to help profile the performance of Spark jobs:
+
+* Cluster-wide monitoring tools, such as [Ganglia](http://ganglia.sourceforge.net/), can provide
+insight into overall cluster utilization and resource bottlenecks. For instance, a Ganglia
+dashboard can quickly reveal whether a particular workload is disk bound, network bound, or
+CPU bound.
+* OS profiling tools such as [dstat](http://dag.wieers.com/home-made/dstat/),
+[iostat](http://linux.die.net/man/1/iostat), and [iotop](http://linux.die.net/man/1/iotop)
+can provide fine-grained profiling on individual nodes.
+* JVM utilities such as `jstack` for providing stack traces, `jmap` for creating heap-dumps,
+`jstat` for reporting time-series statistics and `jconsole` for visually exploring various JVM
+properties are useful for those comfortable with JVM internals.
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 8c33a953a4..6c2336ad0c 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -16,7 +16,7 @@ This guide will show how to use the Spark features described there in Python.
There are a few key differences between the Python and Scala APIs:
* Python is dynamically typed, so RDDs can hold objects of multiple types.
-* PySpark does not yet support a few API calls, such as `lookup`, `sort`, and `persist` at custom storage levels. See the [API docs](api/pyspark/index.html) for details.
+* PySpark does not yet support a few API calls, such as `lookup` and non-text input files, though these will be added in future releases.
In PySpark, RDDs support the same methods as their Scala counterparts but take Python functions and return Python collection types.
Short functions can be passed to RDD methods using Python's [`lambda`](http://www.diveintopython.net/power_of_introspection/lambda_functions.html) syntax:
@@ -53,20 +53,20 @@ In addition, PySpark fully supports interactive use---simply run `./pyspark` to
# Installing and Configuring PySpark
PySpark requires Python 2.6 or higher.
-PySpark jobs are executed using a standard CPython interpreter in order to support Python modules that use C extensions.
+PySpark applications are executed using a standard CPython interpreter in order to support Python modules that use C extensions.
We have not tested PySpark with Python 3 or with alternative Python interpreters, such as [PyPy](http://pypy.org/) or [Jython](http://www.jython.org/).
By default, PySpark requires `python` to be available on the system `PATH` and use it to run programs; an alternate Python executable may be specified by setting the `PYSPARK_PYTHON` environment variable in `conf/spark-env.sh` (or `.cmd` on Windows).
All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported.
-Standalone PySpark jobs should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
+Standalone PySpark applications should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`.
The script automatically adds the `pyspark` package to the `PYTHONPATH`.
# Interactive Use
-The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
+The `pyspark` script launches a Python interpreter that is configured to run PySpark applications. To use `pyspark` interactively, first build Spark, then launch it directly from the command line without any options:
{% highlight bash %}
$ sbt/sbt assembly
@@ -82,7 +82,7 @@ The Python shell can be used explore data interactively and is a simple way to l
>>> help(pyspark) # Show all pyspark functions
{% endhighlight %}
-By default, the `pyspark` shell creates SparkContext that runs jobs locally on a single core.
+By default, the `pyspark` shell creates SparkContext that runs applications locally on a single core.
To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable.
For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
@@ -119,13 +119,13 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env
# Standalone Programs
PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`.
-The Quick Start guide includes a [complete example](quick-start.html#a-standalone-job-in-python) of a standalone Python job.
+The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application.
Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor:
{% highlight python %}
from pyspark import SparkContext
-sc = SparkContext("local", "Job Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg'])
+sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg'])
{% endhighlight %}
Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines.
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 70c3df8095..8f782db5b8 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -6,7 +6,7 @@ title: Quick Start
* This will become a table of contents (this text will be scraped).
{:toc}
-This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone jobs in Scala, Java, and Python.
+This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone applications in Scala, Java, and Python.
See the [programming guide](scala-programming-guide.html) for a more complete reference.
To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run:
@@ -36,7 +36,7 @@ scala> textFile.count() // Number of items in this RDD
res0: Long = 74
scala> textFile.first() // First item in this RDD
-res1: String = # Spark
+res1: String = # Apache Spark
{% endhighlight %}
Now let's use a transformation. We will use the [`filter`](scala-programming-guide.html#transformations) transformation to return a new RDD with a subset of the items in the file.
@@ -101,20 +101,20 @@ res9: Long = 15
It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
-# A Standalone Job in Scala
-Now say we wanted to write a standalone job using the Spark API. We will walk through a simple job in both Scala (with sbt) and Java (with maven). If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
+# A Standalone App in Scala
+Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
-We'll create a very simple Spark job in Scala. So simple, in fact, that it's named `SimpleJob.scala`:
+We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`:
{% highlight scala %}
-/*** SimpleJob.scala ***/
+/*** SimpleApp.scala ***/
import org.apache.spark.SparkContext
import org.apache.spark.SparkContext._
-object SimpleJob {
+object SimpleApp {
def main(args: Array[String]) {
val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system
- val sc = new SparkContext("local", "Simple Job", "YOUR_SPARK_HOME",
+ val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME",
List("target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar"))
val logData = sc.textFile(logFile, 2).cache()
val numAs = logData.filter(line => line.contains("a")).count()
@@ -124,7 +124,7 @@ object SimpleJob {
}
{% endhighlight %}
-This job simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+This program simply counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on:
@@ -146,7 +146,7 @@ If you also wish to read data from Hadoop's HDFS, you will also need to add a de
libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "<your-hdfs-version>"
{% endhighlight %}
-Finally, for sbt to work correctly, we'll need to layout `SimpleJob.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the job's code, then use `sbt run` to execute our example job.
+Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt run` to execute our program.
{% highlight bash %}
$ find .
@@ -155,7 +155,7 @@ $ find .
./src
./src/main
./src/main/scala
-./src/main/scala/SimpleJob.scala
+./src/main/scala/SimpleApp.scala
$ sbt package
$ sbt run
@@ -163,20 +163,20 @@ $ sbt run
Lines with a: 46, Lines with b: 23
{% endhighlight %}
-# A Standalone Job In Java
-Now say we wanted to write a standalone job using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
+# A Standalone App in Java
+Now say we wanted to write a standalone application using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide.
-We'll create a very simple Spark job, `SimpleJob.java`:
+We'll create a very simple Spark application, `SimpleApp.java`:
{% highlight java %}
-/*** SimpleJob.java ***/
+/*** SimpleApp.java ***/
import org.apache.spark.api.java.*;
import org.apache.spark.api.java.function.Function;
-public class SimpleJob {
+public class SimpleApp {
public static void main(String[] args) {
String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system
- JavaSparkContext sc = new JavaSparkContext("local", "Simple Job",
+ JavaSparkContext sc = new JavaSparkContext("local", "Simple App",
"$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"});
JavaRDD<String> logData = sc.textFile(logFile).cache();
@@ -193,9 +193,9 @@ public class SimpleJob {
}
{% endhighlight %}
-This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
+This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail.
-To build the job, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
+To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
{% highlight xml %}
<project>
@@ -238,29 +238,29 @@ $ find .
./src
./src/main
./src/main/java
-./src/main/java/SimpleJob.java
+./src/main/java/SimpleApp.java
{% endhighlight %}
-Now, we can execute the job using Maven:
+Now, we can execute the application using Maven:
{% highlight bash %}
$ mvn package
-$ mvn exec:java -Dexec.mainClass="SimpleJob"
+$ mvn exec:java -Dexec.mainClass="SimpleApp"
...
Lines with a: 46, Lines with b: 23
{% endhighlight %}
-# A Standalone Job In Python
-Now we will show how to write a standalone job using the Python API (PySpark).
+# A Standalone App in Python
+Now we will show how to write a standalone application using the Python API (PySpark).
-As an example, we'll create a simple Spark job, `SimpleJob.py`:
+As an example, we'll create a simple Spark application, `SimpleApp.py`:
{% highlight python %}
-"""SimpleJob.py"""
+"""SimpleApp.py"""
from pyspark import SparkContext
logFile = "$YOUR_SPARK_HOME/README.md" # Should be some file on your system
-sc = SparkContext("local", "Simple job")
+sc = SparkContext("local", "Simple App")
logData = sc.textFile(logFile).cache()
numAs = logData.filter(lambda s: 'a' in s).count()
@@ -270,25 +270,25 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs)
{% endhighlight %}
-This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
+This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed.
As with the Scala and Java examples, we use a SparkContext to create RDDs.
We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference.
-For jobs that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
-`SimpleJob` is simple enough that we do not need to specify any code dependencies.
+For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html).
+`SimpleApp` is simple enough that we do not need to specify any code dependencies.
-We can run this job using the `pyspark` script:
+We can run this application using the `pyspark` script:
{% highlight python %}
$ cd $SPARK_HOME
-$ ./pyspark SimpleJob.py
+$ ./pyspark SimpleApp.py
...
Lines with a: 46, Lines with b: 23
{% endhighlight python %}
-# Running Jobs on a Cluster
+# Running on a Cluster
-There are a few additional considerations when running jobs on a
+There are a few additional considerations when running applicaitons on a
[Spark](spark-standalone.html), [YARN](running-on-yarn.html), or
[Mesos](running-on-mesos.html) cluster.
@@ -306,7 +306,7 @@ your dependent jars one-by-one when creating a SparkContext.
### Setting Configuration Options
Spark includes several configuration options which influence the behavior
-of your job. These should be set as
+of your application. These should be set as
[JVM system properties](configuration.html#system-properties) in your
program. The options will be captured and shipped to all slave nodes.
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index b31f78e8bf..322ff585f1 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -3,24 +3,23 @@ layout: global
title: Running Spark on Mesos
---
-Spark can run on private clusters managed by the [Apache Mesos](http://incubator.apache.org/mesos/) resource manager. Follow the steps below to install Mesos and Spark:
-
-1. Download and build Spark using the instructions [here](index.html).
-2. Download Mesos {{site.MESOS_VERSION}} from a [mirror](http://www.apache.org/dyn/closer.cgi/incubator/mesos/mesos-{{site.MESOS_VERSION}}/).
-3. Configure Mesos using the `configure` script, passing the location of your `JAVA_HOME` using `--with-java-home`. Mesos comes with "template" configure scripts for different platforms, such as `configure.macosx`, that you can run. See the README file in Mesos for other options. **Note:** If you want to run Mesos without installing it into the default paths on your system (e.g. if you don't have administrative privileges to install it), you should also pass the `--prefix` option to `configure` to tell it where to install. For example, pass `--prefix=/home/user/mesos`. By default the prefix is `/usr/local`.
-4. Build Mesos using `make`, and then install it using `make install`.
-5. Create a file called `spark-env.sh` in Spark's `conf` directory, by copying `conf/spark-env.sh.template`, and add the following lines it:
- * `export MESOS_NATIVE_LIBRARY=<path to libmesos.so>`. This path is usually `<prefix>/lib/libmesos.so` (where the prefix is `/usr/local` by default). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `.so`.
-6. Copy Spark and Mesos to the _same_ paths on all the nodes in the cluster (or, for Mesos, `make install` on every node).
-7. Configure Mesos for deployment:
- * On your master node, edit `<prefix>/var/mesos/deploy/masters` to list your master and `<prefix>/var/mesos/deploy/slaves` to list the slaves, where `<prefix>` is the prefix where you installed Mesos (`/usr/local` by default).
- * On all nodes, edit `<prefix>/var/mesos/conf/mesos.conf` and add the line `master=HOST:5050`, where HOST is your master node.
- * Run `<prefix>/sbin/mesos-start-cluster.sh` on your master to start Mesos. If all goes well, you should see Mesos's web UI on port 8080 of the master machine.
- * See Mesos's README file for more information on deploying it.
-8. To run a Spark job against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:5050` as the first parameter, where `HOST` is the machine running your Mesos master. In addition, pass the location of Spark on your nodes as the third parameter, and a list of JAR files containing your JAR's code as the fourth (these will automatically get copied to the workers). For example:
+Spark can run on clusters managed by [Apache Mesos](http://mesos.apache.org/). Follow the steps below to install Mesos and Spark:
+
+1. Download and build Spark using the instructions [here](index.html). **Note:** Don't forget to consider what version of HDFS you might want to use!
+2. Download, build, install, and start Mesos {{site.MESOS_VERSION}} on your cluster. You can download the Mesos distribution from a [mirror](http://www.apache.org/dyn/closer.cgi/mesos/{{site.MESOS_VERSION}}/). See the Mesos [Getting Started](http://mesos.apache.org/gettingstarted) page for more information. **Note:** If you want to run Mesos without installing it into the default paths on your system (e.g., if you don't have administrative privileges to install it), you should also pass the `--prefix` option to `configure` to tell it where to install. For example, pass `--prefix=/home/user/mesos`. By default the prefix is `/usr/local`.
+3. Create a Spark "distribution" using `make-distribution.sh`.
+4. Rename the `dist` directory created from `make-distribution.sh` to `spark-{{site.SPARK_VERSION}}`.
+5. Create a `tar` archive: `tar czf spark-{{site.SPARK_VERSION}}.tar.gz spark-{{site.SPARK_VERSION}}`
+6. Upload this archive to HDFS or another place accessible from Mesos via `http://`, e.g., [Amazon Simple Storage Service](http://aws.amazon.com/s3): `hadoop fs -put spark-{{site.SPARK_VERSION}}.tar.gz /path/to/spark-{{site.SPARK_VERSION}}.tar.gz`
+7. Create a file called `spark-env.sh` in Spark's `conf` directory, by copying `conf/spark-env.sh.template`, and add the following lines to it:
+ * `export MESOS_NATIVE_LIBRARY=<path to libmesos.so>`. This path is usually `<prefix>/lib/libmesos.so` (where the prefix is `/usr/local` by default, see above). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `libmesos.so`.
+ * `export SPARK_EXECUTOR_URI=<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>`.
+ * `export MASTER=mesos://HOST:PORT` where HOST:PORT is the host and port (default: 5050) of your Mesos master (or `zk://...` if using Mesos with ZooKeeper).
+8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the first parameter. In addition, you'll need to set the `spark.executor.uri` property. For example:
{% highlight scala %}
-new SparkContext("mesos://HOST:5050", "My Job Name", "/home/user/spark", List("my-job.jar"))
+System.setProperty("spark.executor.uri", "<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>")
+val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
{% endhighlight %}
If you want to run Spark on Amazon EC2, you can use the Spark [EC2 launch scripts](ec2-scripts.html), which provide an easy way to launch a cluster with Mesos, Spark, and HDFS pre-configured. This will get you a cluster in about five minutes without any configuration on your part.
@@ -28,24 +27,23 @@ If you want to run Spark on Amazon EC2, you can use the Spark [EC2 launch script
# Mesos Run Modes
Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default,
-each Spark task runs as a separate Mesos task. This allows multiple instances of Spark (and other applications) to share
-machines at a very fine granularity, where each job gets more or fewer machines as it ramps up, but it comes with an
-additional overhead in launching each task, which may be inappropriate for low-latency applications that aim for
-sub-second Spark operations (e.g. interactive queries or serving web requests). The coarse-grained mode will instead
+each Spark task runs as a separate Mesos task. This allows multiple instances of Spark (and other frameworks) to share
+machines at a very fine granularity, where each application gets more or fewer machines as it ramps up, but it comes with an
+additional overhead in launching each task, which may be inappropriate for low-latency applications (e.g. interactive queries or serving web requests). The coarse-grained mode will instead
launch only *one* long-running Spark task on each Mesos machine, and dynamically schedule its own "mini-tasks" within
it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration
-of the job.
+of the application.
To run in coarse-grained mode, set the `spark.mesos.coarse` system property to true *before* creating your SparkContext:
{% highlight scala %}
System.setProperty("spark.mesos.coarse", "true")
-val sc = new SparkContext("mesos://HOST:5050", "Job Name", ...)
+val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
{% endhighlight %}
In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default,
-it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just a single
-job at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
+it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just one
+application at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
Again, this must be done *before* initializing a SparkContext.
@@ -53,6 +51,6 @@ Again, this must be done *before* initializing a SparkContext.
You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a separate service on the machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://<namenode>:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI).
-In addition, it is possible to also run Hadoop MapReduce on Mesos, to get better resource isolation and sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each node. Please refer to the Mesos wiki page on [Running Hadoop on Mesos](https://github.com/mesos/mesos/wiki/Running-Hadoop-on-Mesos).
+In addition, it is possible to also run Hadoop MapReduce on Mesos, to get better resource isolation and sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each node. Please refer to [Hadoop on Mesos](https://github.com/mesos/hadoop).
In either case, HDFS runs separately from Hadoop MapReduce, without going through Mesos.
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index fe5334ffdc..2898af0bed 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -29,8 +29,14 @@ If you want to test out the YARN deployment mode, you can use the current Spark
Most of the configs are the same for Spark on YARN as other deploys. See the Configuration page for more information on those. These are configs that are specific to SPARK on YARN.
+Environment variables:
* `SPARK_YARN_USER_ENV`, to add environment variables to the Spark processes launched on YARN. This can be a comma separated list of environment variables, e.g. `SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar"`.
+System Properties:
+* 'spark.yarn.applicationMaster.waitTries', property to set the number of times the ApplicationMaster waits for the the spark master and then also the number of tries it waits for the Spark Context to be intialized. Default is 10.
+* 'spark.yarn.submit.file.replication', the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives.
+* 'spark.yarn.preserve.staging.files', set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them.
+
# Launching Spark on YARN
Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster.
@@ -38,7 +44,7 @@ This would be used to connect to the cluster, write to the dfs and submit jobs t
The command to launch the YARN Client is as follows:
- SPARK_JAR=<SPARK_YARN_JAR_FILE> ./spark-class spark.deploy.yarn.Client \
+ SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
--jar <YOUR_APP_JAR_FILE> \
--class <APP_MAIN_CLASS> \
--args <APP_MAIN_ARGUMENTS> \
@@ -46,18 +52,35 @@ The command to launch the YARN Client is as follows:
--master-memory <MEMORY_FOR_MASTER> \
--worker-memory <MEMORY_PER_WORKER> \
--worker-cores <CORES_PER_WORKER> \
- --queue <queue_name>
+ --name <application_name> \
+ --queue <queue_name> \
+ --addJars <any_local_files_used_in_SparkContext.addJar> \
+ --files <files_for_distributed_cache> \
+ --archives <archives_for_distributed_cache>
For example:
- SPARK_JAR=./yarn/target/spark-yarn-assembly-{{site.SPARK_VERSION}}.jar ./spark-class spark.deploy.yarn.Client \
- --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar \
- --class spark.examples.SparkPi \
- --args yarn-standalone \
- --num-workers 3 \
- --master-memory 4g \
- --worker-memory 2g \
- --worker-cores 1
+ # Build the Spark assembly JAR and the Spark examples JAR
+ $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly
+
+ # Configure logging
+ $ cp conf/log4j.properties.template conf/log4j.properties
+
+ # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example
+ $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
+ ./spark-class org.apache.spark.deploy.yarn.Client \
+ --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
+ --class org.apache.spark.examples.SparkPi \
+ --args yarn-standalone \
+ --num-workers 3 \
+ --master-memory 4g \
+ --worker-memory 2g \
+ --worker-cores 1
+
+ # Examine the output (replace $YARN_APP_ID in the following with the "application identifier" output by the previous command)
+ # (Note: YARN_APP_LOGS_DIR is usually /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version.)
+ $ cat $YARN_APP_LOGS_DIR/$YARN_APP_ID/container*_000001/stdout
+ Pi is roughly 3.13794
The above starts a YARN Client programs which periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running.
@@ -66,3 +89,5 @@ The above starts a YARN Client programs which periodically polls the Application
- When your application instantiates a Spark context it must use a special "yarn-standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "yarn-standalone" as an argument to your program, as shown in the example above.
- We do not requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed.
- The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored.
+- The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN.
+- The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files.
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index f7768e55fc..03647a2ad2 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -87,10 +87,10 @@ For running on YARN, Spark launches an instance of the standalone deploy cluster
### Deploying Code on a Cluster
-If you want to run your job on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code:
+If you want to run your application on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code:
* `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them).
-* `jars`: A list of JAR files on the local machine containing your job's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your job into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
+* `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
If you run `spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed.
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 994a96f2c9..17066ef0dd 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -3,13 +3,24 @@ layout: global
title: Spark Standalone Mode
---
-In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [deploy scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing.
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing.
+
+# Installing Spark Standalone to a Cluster
+
+The easiest way to deploy Spark is by running the `./make-distribution.sh` script to create a binary distribution.
+This distribution can be deployed to any machine with the Java runtime installed; there is no need to install Scala.
+
+The recommended procedure is to deploy and start the master on one node first, get the master spark URL,
+then modify `conf/spark-env.sh` in the `dist/` directory before deploying to all the other nodes.
# Starting a Cluster Manually
You can start a standalone master server by executing:
- ./spark-class org.apache.spark.deploy.master.Master
+ ./bin/start-master.sh
Once started, the master will print out a `spark://HOST:PORT` URL for itself, which you can use to connect workers to it,
or pass as the "master" argument to `SparkContext`. You can also find this URL on
@@ -22,7 +33,7 @@ Similarly, you can start one or more workers and connect them to the master via:
Once you have started a worker, look at the master's web UI ([http://localhost:8080](http://localhost:8080) by default).
You should see the new node listed there, along with its number of CPUs and memory (minus one gigabyte left for the OS).
-Finally, the following configuration options can be passed to the master and worker:
+Finally, the following configuration options can be passed to the master and worker:
<table class="table">
<tr><th style="width:21%">Argument</th><th>Meaning</th></tr>
@@ -40,11 +51,11 @@ Finally, the following configuration options can be passed to the master and wor
</tr>
<tr>
<td><code>-c CORES</code>, <code>--cores CORES</code></td>
- <td>Total CPU cores to allow Spark jobs to use on the machine (default: all available); only on worker</td>
+ <td>Total CPU cores to allow Spark applicatons to use on the machine (default: all available); only on worker</td>
</tr>
<tr>
<td><code>-m MEM</code>, <code>--memory MEM</code></td>
- <td>Total amount of memory to allow Spark jobs to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
+ <td>Total amount of memory to allow Spark applicatons to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker</td>
</tr>
<tr>
<td><code>-d DIR</code>, <code>--work-dir DIR</code></td>
@@ -55,7 +66,7 @@ Finally, the following configuration options can be passed to the master and wor
# Cluster Launch Scripts
-To launch a Spark standalone cluster with the deploy scripts, you need to create a file called `conf/slaves` in your Spark directory, which should contain the hostnames of all the machines where you would like to start Spark workers, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing, you can just put `localhost` in this file.
+To launch a Spark standalone cluster with the launch scripts, you need to create a file called `conf/slaves` in your Spark directory, which should contain the hostnames of all the machines where you would like to start Spark workers, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing, you can just put `localhost` in this file.
Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`:
@@ -90,15 +101,15 @@ You can optionally configure the cluster further by setting environment variable
</tr>
<tr>
<td><code>SPARK_WORKER_DIR</code></td>
- <td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work).</td>
+ <td>Directory to run applications in, which will include both logs and scratch space (default: SPARK_HOME/work).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_CORES</code></td>
- <td>Total number of cores to allow Spark jobs to use on the machine (default: all available cores).</td>
+ <td>Total number of cores to allow Spark applications to use on the machine (default: all available cores).</td>
</tr>
<tr>
<td><code>SPARK_WORKER_MEMORY</code></td>
- <td>Total amount of memory to allow Spark jobs to use on the machine, e.g. <code>1000m</code>, <code>2g</code> (default: total memory minus 1 GB); note that each job's <i>individual</i> memory is configured using its <code>spark.executor.memory</code> property.</td>
+ <td>Total amount of memory to allow Spark applications to use on the machine, e.g. <code>1000m</code>, <code>2g</code> (default: total memory minus 1 GB); note that each application's <i>individual</i> memory is configured using its <code>spark.executor.memory</code> property.</td>
</tr>
<tr>
<td><code>SPARK_WORKER_WEBUI_PORT</code></td>
@@ -125,22 +136,28 @@ You can optionally configure the cluster further by setting environment variable
**Note:** The launch scripts do not currently support Windows. To run a Spark cluster on Windows, start the master and workers by hand.
-# Connecting a Job to the Cluster
+# Connecting an Application to the Cluster
-To run a job on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext`
+To run an application on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext`
constructor](scala-programming-guide.html#initializing-spark).
To run an interactive Spark shell against the cluster, run the following command:
MASTER=spark://IP:PORT ./spark-shell
+Note that if you are running spark-shell from one of the spark cluster machines, the `spark-shell` script will
+automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`.
+
+You can also pass an option `-c <numCores>` to control the number of cores that spark-shell uses on the cluster.
-# Job Scheduling
+# Resource Scheduling
-The standalone cluster mode currently only supports a simple FIFO scheduler across jobs.
-However, to allow multiple concurrent jobs, you can control the maximum number of resources each Spark job will acquire.
-By default, it will acquire *all* the cores in the cluster, which only makes sense if you run just a single
-job at a time. You can cap the number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
+The standalone cluster mode currently only supports a simple FIFO scheduler across applications.
+However, to allow multiple concurrent users, you can control the maximum number of resources each
+application will acquire.
+By default, it will acquire *all* cores in the cluster, which only makes sense if you just run one
+application at a time. You can cap the number of cores using
+`System.setProperty("spark.cores.max", "10")` (for example).
This value must be set *before* initializing your SparkContext.
@@ -155,3 +172,75 @@ In addition, detailed log output for each job is also written to the work direct
You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the same machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://<namenode>:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on).
+
+# High Availability
+
+By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below.
+
+## Standby Masters with ZooKeeper
+
+**Overview**
+
+Utilizing ZooKeeper to provide leader election and some state storage, you can launch multiple Masters in your cluster connected to the same ZooKeeper instance. One will be elected "leader" and the others will remain in standby mode. If the current leader dies, another Master will be elected, recover the old Master's state, and then resume scheduling. The entire recovery process (from the time the the first leader goes down) should take between 1 and 2 minutes. Note that this delay only affects scheduling _new_ applications -- applications that were already running during Master failover are unaffected.
+
+Learn more about getting started with ZooKeeper [here](http://zookeeper.apache.org/doc/trunk/zookeeperStarted.html).
+
+**Configuration**
+
+In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration:
+
+<table class="table">
+ <tr><th style="width:21%">System property</th><th>Meaning</th></tr>
+ <tr>
+ <td><code>spark.deploy.recoveryMode</code></td>
+ <td>Set to ZOOKEEPER to enable standby Master recovery mode (default: NONE).</td>
+ </tr>
+ <tr>
+ <td><code>spark.deploy.zookeeper.url</code></td>
+ <td>The ZooKeeper cluster url (e.g., 192.168.1.100:2181,192.168.1.101:2181).</td>
+ </tr>
+ <tr>
+ <td><code>spark.deploy.zookeeper.dir</code></td>
+ <td>The directory in ZooKeeper to store recovery state (default: /spark).</td>
+ </tr>
+</table>
+
+Possible gotcha: If you have multiple Masters in your cluster but fail to correctly configure the Masters to use ZooKeeper, the Masters will fail to discover each other and think they're all leaders. This will not lead to a healthy cluster state (as all Masters will schedule independently).
+
+**Details**
+
+After you have a ZooKeeper cluster set up, enabling high availability is straightforward. Simply start multiple Master processes on different nodes with the same ZooKeeper configuration (ZooKeeper URL and directory). Masters can be added and removed at any time.
+
+In order to schedule new applications or add Workers to the cluster, they need to know the IP address of the current leader. This can be accomplished by simply passing in a list of Masters where you used to pass in a single one. For example, you might start your SparkContext pointing to ``spark://host1:port1,host2:port2``. This would cause your SparkContext to try registering with both Masters -- if ``host1`` goes down, this configuration would still be correct as we'd find the new leader, ``host2``.
+
+There's an important distinction to be made between "registering with a Master" and normal operation. When starting up, an application or Worker needs to be able to find and register with the current lead Master. Once it successfully registers, though, it is "in the system" (i.e., stored in ZooKeeper). If failover occurs, the new leader will contact all previously registered applications and Workers to inform them of the change in leadership, so they need not even have known of the existence of the new Master at startup.
+
+Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of.
+
+## Single-Node Recovery with Local File System
+
+**Overview**
+
+ZooKeeper is the best way to go for production-level high availability, but if you just want to be able to restart the Master if it goes down, FILESYSTEM mode can take care of it. When applications and Workers register, they have enough state written to the provided directory so that they can be recovered upon a restart of the Master process.
+
+**Configuration**
+
+In order to enable this recovery mode, you can set SPARK_DAEMON_JAVA_OPTS in spark-env using this configuration:
+
+<table class="table">
+ <tr><th style="width:21%">System property</th><th>Meaning</th></tr>
+ <tr>
+ <td><code>spark.deploy.recoveryMode</code></td>
+ <td>Set to FILESYSTEM to enable single-node recovery mode (default: NONE).</td>
+ </tr>
+ <tr>
+ <td><code>spark.deploy.recoveryDirectory</code></td>
+ <td>The directory in which Spark will store recovery state, accessible from the Master's perspective.</td>
+ </tr>
+</table>
+
+**Details**
+
+* This solution can be used in tandem with a process monitor/manager like [monit](http://mmonit.com/monit/), or just to enable manual recovery via restart.
+* While filesystem recovery seems straightforwardly better than not doing any recovery at all, this mode may be suboptimal for certain development or experimental purposes. In particular, killing a master via stop-master.sh does not clean up its recovery state, so whenever you start a new Master, it will enter recovery mode. This could increase the startup time by up to 1 minute if it needs to wait for all previously-registered Workers/clients to timeout.
+* While it's not officially supported, you could mount an NFS directory as the recovery directory. If the original Master node dies completely, you could then start a Master on a different node, which would correctly recover all previously registered Workers/applications (equivalent to ZooKeeper recovery). Future applications will have to be able to find the new Master, however, in order to register.
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index c7df172024..835b257238 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -122,12 +122,12 @@ Spark Streaming features windowed computations, which allow you to apply transfo
<table class="table">
<tr><th style="width:30%">Transformation</th><th>Meaning</th></tr>
<tr>
- <td> <b>window</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+ <td> <b>window</b>(<i>windowDuration</i>, <i>slideDuration</i>) </td>
<td> Return a new DStream which is computed based on windowed batches of the source DStream. <i>windowDuration</i> is the width of the window and <i>slideTime</i> is the frequency during which the window is calculated. Both times must be multiples of the batch interval.
</td>
</tr>
<tr>
- <td> <b>countByWindow</b>(<i>windowDuration</i>, </i>slideDuration</i>) </td>
+ <td> <b>countByWindow</b>(<i>windowDuration</i>, <i>slideDuration</i>) </td>
<td> Return a sliding count of elements in the stream. <i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
@@ -161,7 +161,6 @@ Spark Streaming features windowed computations, which allow you to apply transfo
<i>windowDuration</i> and <i>slideDuration</i> are exactly as defined in <code>window()</code>.
</td>
</tr>
-
</table>
A complete list of DStream operations is available in the API documentation of [DStream](api/streaming/index.html#org.apache.spark.streaming.DStream) and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.PairDStreamFunctions).
diff --git a/docs/tuning.md b/docs/tuning.md
index 28d88a2659..f491ae9b95 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -175,7 +175,7 @@ To further tune garbage collection, we first need to understand some basic infor
* Java Heap space is divided in to two regions Young and Old. The Young generation is meant to hold short-lived objects
while the Old generation is intended for objects with longer lifetimes.
-* The Young generation is further divided into three regions [Eden, Survivor1, Survivor2].
+* The Young generation is further divided into three regions \[Eden, Survivor1, Survivor2\].
* A simplified description of the garbage collection procedure: When Eden is full, a minor GC is run on Eden and objects
that are alive from Eden and Survivor1 are copied to Survivor2. The Survivor regions are swapped. If an object is old
diff --git a/ec2/README b/ec2/README
index 0add81312c..433da37b4c 100644
--- a/ec2/README
+++ b/ec2/README
@@ -1,4 +1,4 @@
This folder contains a script, spark-ec2, for launching Spark clusters on
Amazon EC2. Usage instructions are available online at:
-http://spark-project.org/docs/latest/ec2-scripts.html
+http://spark.incubator.apache.org/docs/latest/ec2-scripts.html
diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
index 675429c57e..42e8faa26e 100644
--- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
+++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
@@ -1,5 +1,22 @@
#!/usr/bin/env bash
+#
+# 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.
+#
+
# These variables are automatically filled in by the spark-ec2 script.
export MASTERS="{{master_list}}"
export SLAVES="{{slave_list}}"
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 932e70db96..65868b76b9 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -23,6 +23,7 @@ from __future__ import with_statement
import logging
import os
+import pipes
import random
import shutil
import subprocess
@@ -36,6 +37,9 @@ import boto
from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType
from boto import ec2
+class UsageError(Exception):
+ pass
+
# A URL prefix from which to fetch AMI information
AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list"
@@ -66,7 +70,7 @@ def parse_args():
"slaves across multiple (an additional $0.01/Gb for bandwidth" +
"between zones applies)")
parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use")
- parser.add_option("-v", "--spark-version", default="0.7.3",
+ parser.add_option("-v", "--spark-version", default="0.8.0",
help="Version of Spark to use: 'X.Y.Z' or a specific git hash")
parser.add_option("--spark-git-repo",
default="https://github.com/mesos/spark",
@@ -103,11 +107,7 @@ def parse_args():
parser.print_help()
sys.exit(1)
(action, cluster_name) = args
- if opts.identity_file == None and action in ['launch', 'login', 'start']:
- print >> stderr, ("ERROR: The -i or --identity-file argument is " +
- "required for " + action)
- sys.exit(1)
-
+
# Boto config check
# http://boto.cloudhackers.com/en/latest/boto_config_tut.html
home_dir = os.getenv('HOME')
@@ -155,7 +155,7 @@ def is_active(instance):
# Return correct versions of Spark and Shark, given the supplied Spark version
def get_spark_shark_version(opts):
- spark_shark_map = {"0.7.3": "0.7.0"}
+ spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0"}
version = opts.spark_version.replace("v", "")
if version not in spark_shark_map:
print >> stderr, "Don't know about Spark version: %s" % version
@@ -215,11 +215,10 @@ def launch_cluster(conn, opts, cluster_name):
master_group.authorize(src_group=slave_group)
master_group.authorize('tcp', 22, 22, '0.0.0.0/0')
master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0')
- master_group.authorize('tcp', 33000, 33000, '0.0.0.0/0')
master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0')
master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0')
master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0')
- master_group.authorize('tcp', 3030, 3035, '0.0.0.0/0')
+ master_group.authorize('tcp', 4040, 4045, '0.0.0.0/0')
if opts.ganglia:
master_group.authorize('tcp', 5080, 5080, '0.0.0.0/0')
if slave_group.rules == []: # Group was just now created
@@ -365,12 +364,12 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
slave_nodes = []
for res in reservations:
active = [i for i in res.instances if is_active(i)]
- if len(active) > 0:
- group_names = [g.name for g in res.groups]
+ for inst in active:
+ group_names = [g.name for g in inst.groups]
if group_names == [cluster_name + "-master"]:
- master_nodes += res.instances
+ master_nodes.append(inst)
elif group_names == [cluster_name + "-slaves"]:
- slave_nodes += res.instances
+ slave_nodes.append(inst)
if any((master_nodes, slave_nodes)):
print ("Found %d master(s), %d slaves" %
(len(master_nodes), len(slave_nodes)))
@@ -391,10 +390,18 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
master = master_nodes[0].public_dns_name
if deploy_ssh_key:
- print "Copying SSH key %s to master..." % opts.identity_file
- ssh(master, opts, 'mkdir -p ~/.ssh')
- scp(master, opts, opts.identity_file, '~/.ssh/id_rsa')
- ssh(master, opts, 'chmod 600 ~/.ssh/id_rsa')
+ print "Generating cluster's SSH key on master..."
+ key_setup = """
+ [ -f ~/.ssh/id_rsa ] ||
+ (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa &&
+ cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys)
+ """
+ ssh(master, opts, key_setup)
+ dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh'])
+ print "Transferring cluster's SSH key to slaves..."
+ for slave in slave_nodes:
+ print slave.public_dns_name
+ ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar)
modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs',
'mapreduce', 'spark-standalone']
@@ -536,18 +543,33 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
dest.write(text)
dest.close()
# rsync the whole directory over to the master machine
- command = (("rsync -rv -e 'ssh -o StrictHostKeyChecking=no -i %s' " +
- "'%s/' '%s@%s:/'") % (opts.identity_file, tmp_dir, opts.user, active_master))
- subprocess.check_call(command, shell=True)
+ command = [
+ 'rsync', '-rv',
+ '-e', stringify_command(ssh_command(opts)),
+ "%s/" % tmp_dir,
+ "%s@%s:/" % (opts.user, active_master)
+ ]
+ subprocess.check_call(command)
# Remove the temp directory we created above
shutil.rmtree(tmp_dir)
-# Copy a file to a given host through scp, throwing an exception if scp fails
-def scp(host, opts, local_file, dest_file):
- subprocess.check_call(
- "scp -q -o StrictHostKeyChecking=no -i %s '%s' '%s@%s:%s'" %
- (opts.identity_file, local_file, opts.user, host, dest_file), shell=True)
+def stringify_command(parts):
+ if isinstance(parts, str):
+ return parts
+ else:
+ return ' '.join(map(pipes.quote, parts))
+
+
+def ssh_args(opts):
+ parts = ['-o', 'StrictHostKeyChecking=no']
+ if opts.identity_file is not None:
+ parts += ['-i', opts.identity_file]
+ return parts
+
+
+def ssh_command(opts):
+ return ['ssh'] + ssh_args(opts)
# Run a command on a host through ssh, retrying up to two times
@@ -557,18 +579,42 @@ def ssh(host, opts, command):
while True:
try:
return subprocess.check_call(
- "ssh -t -o StrictHostKeyChecking=no -i %s %s@%s '%s'" %
- (opts.identity_file, opts.user, host, command), shell=True)
+ ssh_command(opts) + ['-t', '%s@%s' % (opts.user, host), stringify_command(command)])
except subprocess.CalledProcessError as e:
if (tries > 2):
- raise e
- print "Couldn't connect to host {0}, waiting 30 seconds".format(e)
+ # If this was an ssh failure, provide the user with hints.
+ if e.returncode == 255:
+ raise UsageError("Failed to SSH to remote host {0}.\nPlease check that you have provided the correct --identity-file and --key-pair parameters and try again.".format(host))
+ else:
+ raise e
+ print >> stderr, "Error executing remote command, retrying after 30 seconds: {0}".format(e)
time.sleep(30)
tries = tries + 1
+def ssh_read(host, opts, command):
+ return subprocess.check_output(
+ ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)])
+def ssh_write(host, opts, command, input):
+ tries = 0
+ while True:
+ proc = subprocess.Popen(
+ ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)],
+ stdin=subprocess.PIPE)
+ proc.stdin.write(input)
+ proc.stdin.close()
+ status = proc.wait()
+ if status == 0:
+ break
+ elif (tries > 2):
+ raise RuntimeError("ssh_write failed with error %s" % proc.returncode)
+ else:
+ print >> stderr, "Error {0} while executing remote command, retrying after 30 seconds".format(status)
+ time.sleep(30)
+ tries = tries + 1
+
# Gets a list of zones to launch instances in
def get_zones(conn, opts):
@@ -587,7 +633,7 @@ def get_partition(total, num_partitions, current_partitions):
return num_slaves_this_zone
-def main():
+def real_main():
(opts, action, cluster_name) = parse_args()
try:
conn = ec2.connect_to_region(opts.region)
@@ -670,11 +716,11 @@ def main():
conn, opts, cluster_name)
master = master_nodes[0].public_dns_name
print "Logging into master " + master + "..."
- proxy_opt = ""
+ proxy_opt = []
if opts.proxy_port != None:
- proxy_opt = "-D " + opts.proxy_port
- subprocess.check_call("ssh -o StrictHostKeyChecking=no -i %s %s %s@%s" %
- (opts.identity_file, proxy_opt, opts.user, master), shell=True)
+ proxy_opt = ['-D', opts.proxy_port]
+ subprocess.check_call(
+ ssh_command(opts) + proxy_opt + ['-t', "%s@%s" % (opts.user, master)])
elif action == "get-master":
(master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name)
@@ -716,6 +762,13 @@ def main():
sys.exit(1)
+def main():
+ try:
+ real_main()
+ except UsageError, e:
+ print >> stderr, "\nError:\n", e
+
+
if __name__ == "__main__":
logging.basicConfig()
main()
diff --git a/examples/pom.xml b/examples/pom.xml
index 224cf6c96c..b8c020a321 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -21,38 +21,46 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-examples</artifactId>
+ <artifactId>spark-examples_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project Examples</name>
<url>http://spark.incubator.apache.org/</url>
+ <repositories>
+ <!-- A repository in the local filesystem for the Kafka JAR, which we modified for Scala 2.9 -->
+ <repository>
+ <id>lib</id>
+ <url>file://${project.basedir}/lib</url>
+ </repository>
+ </repositories>
+
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming</artifactId>
+ <artifactId>spark-streaming_2.9.3</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib</artifactId>
+ <artifactId>spark-mllib_2.9.3</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel</artifactId>
+ <artifactId>spark-bagel_2.9.3</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
@@ -72,6 +80,12 @@
</exclusions>
</dependency>
<dependency>
+ <groupId>org.apache.kafka</groupId>
+ <artifactId>kafka</artifactId>
+ <version>0.7.2-spark</version> <!-- Comes from our in-project repository -->
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
</dependency>
@@ -82,12 +96,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -127,20 +141,6 @@
</dependency>
</dependencies>
- <profiles>
- <profile>
- <id>hadoop2-yarn</id>
- <dependencies>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn</artifactId>
- <version>${project.version}</version>
- <scope>provided</scope>
- </dependency>
- </dependencies>
- </profile>
- </profiles>
-
<build>
<outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
@@ -175,7 +175,7 @@
</goals>
<configuration>
<transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>reference.conf</resource>
</transformer>
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
index f7bf75b4e5..bc2db39c12 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -21,8 +21,6 @@ import java.util.Random
import org.apache.spark.SparkContext
import org.apache.spark.util.Vector
import org.apache.spark.SparkContext._
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
/**
* K-means clustering.
diff --git a/make-distribution.sh b/make-distribution.sh
index 91f6278491..32bbdb90a5 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -95,7 +95,7 @@ cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/jars/"
# Copy other things
mkdir "$DISTDIR"/conf
-cp -r "$FWDIR/conf/*.template" "$DISTDIR"
+cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
cp -r "$FWDIR/bin" "$DISTDIR"
cp -r "$FWDIR/python" "$DISTDIR"
cp "$FWDIR/spark-class" "$DISTDIR"
diff --git a/mllib/data/sample_svm_data.txt b/mllib/data/sample_svm_data.txt
new file mode 100644
index 0000000000..7ab30bd93c
--- /dev/null
+++ b/mllib/data/sample_svm_data.txt
@@ -0,0 +1,322 @@
+1 0 2.52078447201548 0 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 12.72816758217773 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 0 0 0 0 4.745052855503306 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 0 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 0 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 0 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 0 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 0 0 2.004684436494304 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 6.857275130999357 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 0 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 10.4087817597473 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 0 0 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 12.72816758217773 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 4.745052855503306 0 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+1 0 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 0 2.52078447201548 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+0 0 2.52078447201548 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 2.061393766919624 0 0 2.004684436494304 0 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 4.745052855503306 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 2.52078447201548 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 0 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 0 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 0 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 6.857275130999357 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 4.745052855503306 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 0 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 4.745052855503306 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 0 0 0 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+0 0 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 2.061393766919624 2.619965104088255 4.745052855503306 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 0 2.52078447201548 0 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 0 0 0 0 0
+0 0 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 0 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 6.857275130999357 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+0 0 2.52078447201548 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 0 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 0 2.52078447201548 0 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 0 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 0 2.52078447201548 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 4.745052855503306 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 0 0 0 0 0 0 0 2.122974378789621 0 0 0 0 12.72816758217773 10.4087817597473 12.72816758217773 17.97228742438751
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 4.745052855503306 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 0 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 0 2.52078447201548 2.061393766919624 0 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 0 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 4.745052855503306 2.004684436494304 0 0 0 0 0 0 0 0 0 0
+1 0 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 0 0 0 6.857275130999357 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 4.745052855503306 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 6.857275130999357 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 0 4.745052855503306 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 0 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 4.745052855503306 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+0 0 2.52078447201548 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 4.745052855503306 2.004684436494304 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 0 2.52078447201548 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 0 0 0 2.000347299268466 2.122974378789621 0 0 6.857275130999357 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 4.745052855503306 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 2.619965104088255 0 0 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 0 0 2.061393766919624 0 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 2.061393766919624 0 0 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 6.857275130999357 0 0 0 0 0
+1 2.857738033247042 0 2.061393766919624 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 2.000347299268466 0 0 0 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 2.52078447201548 2.061393766919624 2.619965104088255 0 0 2.000347299268466 0 0 0 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 0 0 0 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 4.745052855503306 0 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 2.52078447201548 0 0 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 2.857738033247042 2.52078447201548 0 2.619965104088255 0 2.004684436494304 0 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 2.061393766919624 0 0 2.004684436494304 0 2.122974378789621 0 0 0 0 0 10.4087817597473 0 0
+0 2.857738033247042 0 2.061393766919624 2.619965104088255 0 2.004684436494304 0 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+0 2.857738033247042 0 0 2.619965104088255 0 2.004684436494304 2.000347299268466 2.122974378789621 0 0 0 0 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 2.000347299268466 0 2.228387042742021 2.228387042742023 0 0 0 0 0 0
+1 0 2.52078447201548 0 2.619965104088255 0 0 0 0 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
+1 2.857738033247042 0 0 2.619965104088255 0 0 0 2.122974378789621 2.228387042742021 2.228387042742023 0 2.055002875864414 0 0 0 0
diff --git a/mllib/pom.xml b/mllib/pom.xml
index 966caf6835..f472082ad1 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -21,12 +21,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib</artifactId>
+ <artifactId>spark-mllib_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project ML Library</name>
<url>http://spark.incubator.apache.org/</url>
@@ -34,7 +34,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -48,12 +48,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
index 3511e24bce..3b8f8550d0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
@@ -29,7 +29,7 @@ import org.apache.spark.mllib.util.DataValidators
import org.jblas.DoubleMatrix
/**
- * Model built using SVM.
+ * Model for Support Vector Machines (SVMs).
*
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
@@ -48,8 +48,8 @@ class SVMModel(
}
/**
- * Train an SVM using Stochastic Gradient Descent.
- * NOTE: Labels used in SVM should be {0, 1}
+ * Train a Support Vector Machine (SVM) using Stochastic Gradient Descent.
+ * NOTE: Labels used in SVM should be {0, 1}.
*/
class SVMWithSGD private (
var stepSize: Double,
@@ -79,7 +79,7 @@ class SVMWithSGD private (
}
/**
- * Top-level methods for calling SVM. NOTE: Labels used in SVM should be {0, 1}
+ * Top-level methods for calling SVM. NOTE: Labels used in SVM should be {0, 1}.
*/
object SVMWithSGD {
@@ -88,14 +88,15 @@ object SVMWithSGD {
* of iterations of gradient descent using the specified step size. Each iteration uses
* `miniBatchFraction` fraction of the data to calculate the gradient. The weights used in
* gradient descent are initialized using the initial weights provided.
- * NOTE: Labels used in SVM should be {0, 1}
+ *
+ * NOTE: Labels used in SVM should be {0, 1}.
*
* @param input RDD of (label, array of features) pairs.
* @param numIterations Number of iterations of gradient descent to run.
* @param stepSize Step size to be used for each iteration of gradient descent.
* @param regParam Regularization parameter.
* @param miniBatchFraction Fraction of data to be used per iteration.
- * @param initialWeights Initial set of weights to be used. Array should be equal in size to
+ * @param initialWeights Initial set of weights to be used. Array should be equal in size to
* the number of features in the data.
*/
def train(
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index be002d02bc..36853acab5 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -21,7 +21,8 @@ import scala.collection.mutable.{ArrayBuffer, BitSet}
import scala.util.Random
import scala.util.Sorting
-import org.apache.spark.{HashPartitioner, Partitioner, SparkContext}
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.{Logging, HashPartitioner, Partitioner, SparkContext}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.rdd.RDD
import org.apache.spark.serializer.KryoRegistrator
@@ -61,6 +62,12 @@ case class Rating(val user: Int, val product: Int, val rating: Double)
/**
* Alternating Least Squares matrix factorization.
*
+ * ALS attempts to estimate the ratings matrix `R` as the product of two lower-rank matrices,
+ * `X` and `Y`, i.e. `Xt * Y = R`. Typically these approximations are called 'factor' matrices.
+ * The general approach is iterative. During each iteration, one of the factor matrices is held
+ * constant, while the other is solved for using least squares. The newly-solved factor matrix is
+ * then held constant while solving for the other factor matrix.
+ *
* This is a blocked implementation of the ALS factorization algorithm that groups the two sets
* of factors (referred to as "users" and "products") into blocks and reduces communication by only
* sending one copy of each user vector to each product block on each iteration, and only for the
@@ -70,11 +77,21 @@ case class Rating(val user: Int, val product: Int, val rating: Double)
* vectors it receives from each user block it will depend on). This allows us to send only an
* array of feature vectors between each user block and product block, and have the product block
* find the users' ratings and update the products based on these messages.
+ *
+ * For implicit preference data, the algorithm used is based on
+ * "Collaborative Filtering for Implicit Feedback Datasets", available at
+ * [[http://research.yahoo.com/pub/2433]], adapted for the blocked approach used here.
+ *
+ * Essentially instead of finding the low-rank approximations to the rating matrix `R`,
+ * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if r > 0
+ * and 0 if r = 0. The ratings then act as 'confidence' values related to strength of indicated user
+ * preferences rather than explicit ratings given to items.
*/
-class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double)
- extends Serializable
+class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double,
+ var implicitPrefs: Boolean, var alpha: Double)
+ extends Serializable with Logging
{
- def this() = this(-1, 10, 10, 0.01)
+ def this() = this(-1, 10, 10, 0.01, false, 1.0)
/**
* Set the number of blocks to parallelize the computation into; pass -1 for an auto-configured
@@ -103,6 +120,16 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
this
}
+ def setImplicitPrefs(implicitPrefs: Boolean): ALS = {
+ this.implicitPrefs = implicitPrefs
+ this
+ }
+
+ def setAlpha(alpha: Double): ALS = {
+ this.alpha = alpha
+ this
+ }
+
/**
* Run ALS with the configured parameters on an input RDD of (user, product, rating) triples.
* Returns a MatrixFactorizationModel with feature vectors for each user and product.
@@ -147,19 +174,24 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
}
}
- for (iter <- 0 until iterations) {
+ for (iter <- 1 to iterations) {
// perform ALS update
- products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda)
- users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda)
+ logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations))
+ // YtY / XtX is an Option[DoubleMatrix] and is only required for the implicit feedback model
+ val YtY = computeYtY(users)
+ val YtYb = ratings.context.broadcast(YtY)
+ products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda,
+ alpha, YtYb)
+ logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations))
+ val XtX = computeYtY(products)
+ val XtXb = ratings.context.broadcast(XtX)
+ users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda,
+ alpha, XtXb)
}
// Flatten and cache the two final RDDs to un-block them
- val usersOut = users.join(userOutLinks).flatMap { case (b, (factors, outLinkBlock)) =>
- for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i))
- }
- val productsOut = products.join(productOutLinks).flatMap { case (b, (factors, outLinkBlock)) =>
- for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i))
- }
+ val usersOut = unblockFactors(users, userOutLinks)
+ val productsOut = unblockFactors(products, productOutLinks)
usersOut.persist()
productsOut.persist()
@@ -168,6 +200,40 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
}
/**
+ * Computes the (`rank x rank`) matrix `YtY`, where `Y` is the (`nui x rank`) matrix of factors
+ * for each user (or product), in a distributed fashion. Here `reduceByKeyLocally` is used as
+ * the driver program requires `YtY` to broadcast it to the slaves
+ * @param factors the (block-distributed) user or product factor vectors
+ * @return Option[YtY] - whose value is only used in the implicit preference model
+ */
+ def computeYtY(factors: RDD[(Int, Array[Array[Double]])]) = {
+ if (implicitPrefs) {
+ Option(
+ factors.flatMapValues{ case factorArray =>
+ factorArray.map{ vector =>
+ val x = new DoubleMatrix(vector)
+ x.mmul(x.transpose())
+ }
+ }.reduceByKeyLocally((a, b) => a.addi(b))
+ .values
+ .reduce((a, b) => a.addi(b))
+ )
+ } else {
+ None
+ }
+ }
+
+ /**
+ * Flatten out blocked user or product factors into an RDD of (id, factor vector) pairs
+ */
+ def unblockFactors(blockedFactors: RDD[(Int, Array[Array[Double]])],
+ outLinks: RDD[(Int, OutLinkBlock)]) = {
+ blockedFactors.join(outLinks).flatMap{ case (b, (factors, outLinkBlock)) =>
+ for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i))
+ }
+ }
+
+ /**
* Make the out-links table for a block of the users (or products) dataset given the list of
* (user, product, rating) values for the users in that block (or the opposite for products).
*/
@@ -251,7 +317,9 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
userInLinks: RDD[(Int, InLinkBlock)],
partitioner: Partitioner,
rank: Int,
- lambda: Double)
+ lambda: Double,
+ alpha: Double,
+ YtY: Broadcast[Option[DoubleMatrix]])
: RDD[(Int, Array[Array[Double]])] =
{
val numBlocks = products.partitions.size
@@ -265,7 +333,9 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
toSend.zipWithIndex.map{ case (buf, idx) => (idx, (bid, buf.toArray)) }
}.groupByKey(partitioner)
.join(userInLinks)
- .mapValues{ case (messages, inLinkBlock) => updateBlock(messages, inLinkBlock, rank, lambda) }
+ .mapValues{ case (messages, inLinkBlock) =>
+ updateBlock(messages, inLinkBlock, rank, lambda, alpha, YtY)
+ }
}
/**
@@ -273,7 +343,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
* it received from each product and its InLinkBlock.
*/
def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock,
- rank: Int, lambda: Double)
+ rank: Int, lambda: Double, alpha: Double, YtY: Broadcast[Option[DoubleMatrix]])
: Array[Array[Double]] =
{
// Sort the incoming block factor messages by block ID and make them an array
@@ -298,8 +368,14 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
fillXtX(x, tempXtX)
val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p)
for (i <- 0 until us.length) {
- userXtX(us(i)).addi(tempXtX)
- SimpleBlas.axpy(rs(i), x, userXy(us(i)))
+ implicitPrefs match {
+ case false =>
+ userXtX(us(i)).addi(tempXtX)
+ SimpleBlas.axpy(rs(i), x, userXy(us(i)))
+ case true =>
+ userXtX(us(i)).addi(tempXtX.mul(alpha * rs(i)))
+ SimpleBlas.axpy(1 + alpha * rs(i), x, userXy(us(i)))
+ }
}
}
}
@@ -311,7 +387,10 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
// Add regularization
(0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda)
// Solve the resulting matrix, which is symmetric and positive-definite
- Solve.solvePositive(fullXtX, userXy(index)).data
+ implicitPrefs match {
+ case false => Solve.solvePositive(fullXtX, userXy(index)).data
+ case true => Solve.solvePositive(fullXtX.add(YtY.value.get), userXy(index)).data
+ }
}
}
@@ -381,7 +460,7 @@ object ALS {
blocks: Int)
: MatrixFactorizationModel =
{
- new ALS(blocks, rank, iterations, lambda).run(ratings)
+ new ALS(blocks, rank, iterations, lambda, false, 1.0).run(ratings)
}
/**
@@ -419,6 +498,68 @@ object ALS {
train(ratings, rank, iterations, 0.01, -1)
}
+ /**
+ * Train a matrix factorization model given an RDD of 'implicit preferences' given by users
+ * to some products, in the form of (userID, productID, preference) pairs. We approximate the
+ * ratings matrix as the product of two lower-rank matrices of a given rank (number of features).
+ * To solve for these features, we run a given number of iterations of ALS. This is done using
+ * a level of parallelism given by `blocks`.
+ *
+ * @param ratings RDD of (userID, productID, rating) pairs
+ * @param rank number of features to use
+ * @param iterations number of iterations of ALS (recommended: 10-20)
+ * @param lambda regularization factor (recommended: 0.01)
+ * @param blocks level of parallelism to split computation into
+ * @param alpha confidence parameter (only applies when immplicitPrefs = true)
+ */
+ def trainImplicit(
+ ratings: RDD[Rating],
+ rank: Int,
+ iterations: Int,
+ lambda: Double,
+ blocks: Int,
+ alpha: Double)
+ : MatrixFactorizationModel =
+ {
+ new ALS(blocks, rank, iterations, lambda, true, alpha).run(ratings)
+ }
+
+ /**
+ * Train a matrix factorization model given an RDD of 'implicit preferences' given by users to
+ * some products, in the form of (userID, productID, preference) pairs. We approximate the
+ * ratings matrix as the product of two lower-rank matrices of a given rank (number of features).
+ * To solve for these features, we run a given number of iterations of ALS. The level of
+ * parallelism is determined automatically based on the number of partitions in `ratings`.
+ *
+ * @param ratings RDD of (userID, productID, rating) pairs
+ * @param rank number of features to use
+ * @param iterations number of iterations of ALS (recommended: 10-20)
+ * @param lambda regularization factor (recommended: 0.01)
+ */
+ def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double)
+ : MatrixFactorizationModel =
+ {
+ trainImplicit(ratings, rank, iterations, lambda, -1, alpha)
+ }
+
+ /**
+ * Train a matrix factorization model given an RDD of 'implicit preferences' ratings given by
+ * users to some products, in the form of (userID, productID, rating) pairs. We approximate the
+ * ratings matrix as the product of two lower-rank matrices of a given rank (number of features).
+ * To solve for these features, we run a given number of iterations of ALS. The level of
+ * parallelism is determined automatically based on the number of partitions in `ratings`.
+ * Model parameters `alpha` and `lambda` are set to reasonable default values
+ *
+ * @param ratings RDD of (userID, productID, rating) pairs
+ * @param rank number of features to use
+ * @param iterations number of iterations of ALS (recommended: 10-20)
+ */
+ def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int)
+ : MatrixFactorizationModel =
+ {
+ trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0)
+ }
+
private class ALSRegistrator extends KryoRegistrator {
override def registerClasses(kryo: Kryo) {
kryo.register(classOf[Rating])
@@ -426,29 +567,37 @@ object ALS {
}
def main(args: Array[String]) {
- if (args.length != 5 && args.length != 6) {
- println("Usage: ALS <master> <ratings_file> <rank> <iterations> <output_dir> [<blocks>]")
+ if (args.length < 5 || args.length > 9) {
+ println("Usage: ALS <master> <ratings_file> <rank> <iterations> <output_dir> " +
+ "[<lambda>] [<implicitPrefs>] [<alpha>] [<blocks>]")
System.exit(1)
}
val (master, ratingsFile, rank, iters, outputDir) =
(args(0), args(1), args(2).toInt, args(3).toInt, args(4))
- val blocks = if (args.length == 6) args(5).toInt else -1
+ val lambda = if (args.length >= 6) args(5).toDouble else 0.01
+ val implicitPrefs = if (args.length >= 7) args(6).toBoolean else false
+ val alpha = if (args.length >= 8) args(7).toDouble else 1
+ val blocks = if (args.length == 9) args(8).toInt else -1
+
System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
System.setProperty("spark.kryo.registrator", classOf[ALSRegistrator].getName)
System.setProperty("spark.kryo.referenceTracking", "false")
System.setProperty("spark.kryoserializer.buffer.mb", "8")
System.setProperty("spark.locality.wait", "10000")
+
val sc = new SparkContext(master, "ALS")
val ratings = sc.textFile(ratingsFile).map { line =>
val fields = line.split(',')
Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble)
}
- val model = ALS.train(ratings, rank, iters, 0.01, blocks)
+ val model = new ALS(rank = rank, iterations = iters, lambda = lambda,
+ numBlocks = blocks, implicitPrefs = implicitPrefs, alpha = alpha).run(ratings)
+
model.userFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") }
.saveAsTextFile(outputDir + "/userFeatures")
model.productFeatures.map{ case (id, vec) => id + "," + vec.mkString(" ") }
.saveAsTextFile(outputDir + "/productFeatures")
println("Final user/product features written to " + outputDir)
- System.exit(0)
+ sc.stop()
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
index ae95ea24fc..597d55e0bb 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
@@ -43,7 +43,7 @@ class LinearRegressionModel(
}
/**
- * Train a regression model with no regularization using Stochastic Gradient Descent.
+ * Train a linear regression model with no regularization using Stochastic Gradient Descent.
*/
class LinearRegressionWithSGD private (
var stepSize: Double,
@@ -83,7 +83,7 @@ object LinearRegressionWithSGD {
* @param numIterations Number of iterations of gradient descent to run.
* @param stepSize Step size to be used for each iteration of gradient descent.
* @param miniBatchFraction Fraction of data to be used per iteration.
- * @param initialWeights Initial set of weights to be used. Array should be equal in size to
+ * @param initialWeights Initial set of weights to be used. Array should be equal in size to
* the number of features in the data.
*/
def train(
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..eafee060cd 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
@@ -19,6 +19,7 @@ package org.apache.spark.mllib.recommendation;
import java.io.Serializable;
import java.util.List;
+import java.lang.Math;
import scala.Tuple2;
@@ -48,7 +49,7 @@ public class JavaALSSuite implements Serializable {
}
void validatePrediction(MatrixFactorizationModel model, int users, int products, int features,
- DoubleMatrix trueRatings, double matchThreshold) {
+ DoubleMatrix trueRatings, double matchThreshold, boolean implicitPrefs, DoubleMatrix truePrefs) {
DoubleMatrix predictedU = new DoubleMatrix(users, features);
List<scala.Tuple2<Object, double[]>> userFeatures = model.userFeatures().toJavaRDD().collect();
for (int i = 0; i < features; ++i) {
@@ -68,12 +69,32 @@ public class JavaALSSuite implements Serializable {
DoubleMatrix predictedRatings = predictedU.mmul(predictedP.transpose());
- for (int u = 0; u < users; ++u) {
- for (int p = 0; p < products; ++p) {
- double prediction = predictedRatings.get(u, p);
- double correct = trueRatings.get(u, p);
- Assert.assertTrue(Math.abs(prediction - correct) < matchThreshold);
+ if (!implicitPrefs) {
+ for (int u = 0; u < users; ++u) {
+ for (int p = 0; p < products; ++p) {
+ double prediction = predictedRatings.get(u, p);
+ double correct = trueRatings.get(u, p);
+ Assert.assertTrue(String.format("Prediction=%2.4f not below match threshold of %2.2f",
+ prediction, matchThreshold), Math.abs(prediction - correct) < matchThreshold);
+ }
}
+ } else {
+ // For implicit prefs we use the confidence-weighted RMSE to test (ref Mahout's implicit ALS tests)
+ double sqErr = 0.0;
+ double denom = 0.0;
+ for (int u = 0; u < users; ++u) {
+ for (int p = 0; p < products; ++p) {
+ double prediction = predictedRatings.get(u, p);
+ double truePref = truePrefs.get(u, p);
+ double confidence = 1.0 + /* alpha = */ 1.0 * trueRatings.get(u, p);
+ double err = confidence * (truePref - prediction) * (truePref - prediction);
+ sqErr += err;
+ denom += 1.0;
+ }
+ }
+ double rmse = Math.sqrt(sqErr / denom);
+ Assert.assertTrue(String.format("Confidence-weighted RMSE=%2.4f above threshold of %2.2f",
+ rmse, matchThreshold), Math.abs(rmse) < matchThreshold);
}
}
@@ -81,30 +102,62 @@ public class JavaALSSuite implements Serializable {
public void runALSUsingStaticMethods() {
int features = 1;
int iterations = 15;
- int users = 10;
- int products = 10;
- scala.Tuple2<List<Rating>, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList(
- users, products, features, 0.7);
+ int users = 50;
+ int products = 100;
+ scala.Tuple3<List<Rating>, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList(
+ users, products, features, 0.7, false);
JavaRDD<Rating> data = sc.parallelize(testData._1());
MatrixFactorizationModel model = ALS.train(data.rdd(), features, iterations);
- validatePrediction(model, users, products, features, testData._2(), 0.3);
+ validatePrediction(model, users, products, features, testData._2(), 0.3, false, testData._3());
}
@Test
public void runALSUsingConstructor() {
int features = 2;
int iterations = 15;
- int users = 20;
- int products = 30;
- scala.Tuple2<List<Rating>, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList(
- users, products, features, 0.7);
+ int users = 100;
+ int products = 200;
+ scala.Tuple3<List<Rating>, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList(
+ users, products, features, 0.7, false);
JavaRDD<Rating> data = sc.parallelize(testData._1());
MatrixFactorizationModel model = new ALS().setRank(features)
.setIterations(iterations)
.run(data.rdd());
- validatePrediction(model, users, products, features, testData._2(), 0.3);
+ validatePrediction(model, users, products, features, testData._2(), 0.3, false, testData._3());
+ }
+
+ @Test
+ public void runImplicitALSUsingStaticMethods() {
+ int features = 1;
+ int iterations = 15;
+ int users = 80;
+ int products = 160;
+ scala.Tuple3<List<Rating>, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList(
+ users, products, features, 0.7, true);
+
+ JavaRDD<Rating> data = sc.parallelize(testData._1());
+ MatrixFactorizationModel model = ALS.trainImplicit(data.rdd(), features, iterations);
+ validatePrediction(model, users, products, features, testData._2(), 0.4, true, testData._3());
+ }
+
+ @Test
+ public void runImplicitALSUsingConstructor() {
+ int features = 2;
+ int iterations = 15;
+ int users = 100;
+ int products = 200;
+ scala.Tuple3<List<Rating>, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList(
+ users, products, features, 0.7, true);
+
+ JavaRDD<Rating> data = sc.parallelize(testData._1());
+
+ MatrixFactorizationModel model = new ALS().setRank(features)
+ .setIterations(iterations)
+ .setImplicitPrefs(true)
+ .run(data.rdd());
+ validatePrediction(model, users, products, features, testData._2(), 0.4, true, testData._3());
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
index 347ef238f4..fafc5ec5f2 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
@@ -34,16 +34,19 @@ object ALSSuite {
users: Int,
products: Int,
features: Int,
- samplingRate: Double): (java.util.List[Rating], DoubleMatrix) = {
- val (sampledRatings, trueRatings) = generateRatings(users, products, features, samplingRate)
- (seqAsJavaList(sampledRatings), trueRatings)
+ samplingRate: Double,
+ implicitPrefs: Boolean): (java.util.List[Rating], DoubleMatrix, DoubleMatrix) = {
+ val (sampledRatings, trueRatings, truePrefs) =
+ generateRatings(users, products, features, samplingRate, implicitPrefs)
+ (seqAsJavaList(sampledRatings), trueRatings, truePrefs)
}
def generateRatings(
users: Int,
products: Int,
features: Int,
- samplingRate: Double): (Seq[Rating], DoubleMatrix) = {
+ samplingRate: Double,
+ implicitPrefs: Boolean = false): (Seq[Rating], DoubleMatrix, DoubleMatrix) = {
val rand = new Random(42)
// Create a random matrix with uniform values from -1 to 1
@@ -52,14 +55,20 @@ object ALSSuite {
val userMatrix = randomMatrix(users, features)
val productMatrix = randomMatrix(features, products)
- val trueRatings = userMatrix.mmul(productMatrix)
+ val (trueRatings, truePrefs) = implicitPrefs match {
+ case true =>
+ val raw = new DoubleMatrix(users, products, Array.fill(users * products)(rand.nextInt(10).toDouble): _*)
+ val prefs = new DoubleMatrix(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0): _*)
+ (raw, prefs)
+ case false => (userMatrix.mmul(productMatrix), null)
+ }
val sampledRatings = {
for (u <- 0 until users; p <- 0 until products if rand.nextDouble() < samplingRate)
yield Rating(u, p, trueRatings.get(u, p))
}
- (sampledRatings, trueRatings)
+ (sampledRatings, trueRatings, truePrefs)
}
}
@@ -78,11 +87,19 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
}
test("rank-1 matrices") {
- testALS(10, 20, 1, 15, 0.7, 0.3)
+ testALS(50, 100, 1, 15, 0.7, 0.3)
}
test("rank-2 matrices") {
- testALS(20, 30, 2, 15, 0.7, 0.3)
+ testALS(100, 200, 2, 15, 0.7, 0.3)
+ }
+
+ test("rank-1 matrices implicit") {
+ testALS(80, 160, 1, 15, 0.7, 0.4, true)
+ }
+
+ test("rank-2 matrices implicit") {
+ testALS(100, 200, 2, 15, 0.7, 0.4, true)
}
/**
@@ -96,11 +113,14 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
* @param matchThreshold max difference allowed to consider a predicted rating correct
*/
def testALS(users: Int, products: Int, features: Int, iterations: Int,
- samplingRate: Double, matchThreshold: Double)
+ samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false)
{
- val (sampledRatings, trueRatings) = ALSSuite.generateRatings(users, products,
- features, samplingRate)
- val model = ALS.train(sc.parallelize(sampledRatings), features, iterations)
+ val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products,
+ features, samplingRate, implicitPrefs)
+ val model = implicitPrefs match {
+ case false => ALS.train(sc.parallelize(sampledRatings), features, iterations)
+ case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations)
+ }
val predictedU = new DoubleMatrix(users, features)
for ((u, vec) <- model.userFeatures.collect(); i <- 0 until features) {
@@ -112,12 +132,31 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
}
val predictedRatings = predictedU.mmul(predictedP.transpose)
- for (u <- 0 until users; p <- 0 until products) {
- val prediction = predictedRatings.get(u, p)
- val correct = trueRatings.get(u, p)
- if (math.abs(prediction - correct) > matchThreshold) {
- fail("Model failed to predict (%d, %d): %f vs %f\ncorr: %s\npred: %s\nU: %s\n P: %s".format(
- u, p, correct, prediction, trueRatings, predictedRatings, predictedU, predictedP))
+ if (!implicitPrefs) {
+ for (u <- 0 until users; p <- 0 until products) {
+ val prediction = predictedRatings.get(u, p)
+ val correct = trueRatings.get(u, p)
+ if (math.abs(prediction - correct) > matchThreshold) {
+ fail("Model failed to predict (%d, %d): %f vs %f\ncorr: %s\npred: %s\nU: %s\n P: %s".format(
+ u, p, correct, prediction, trueRatings, predictedRatings, predictedU, predictedP))
+ }
+ }
+ } else {
+ // For implicit prefs we use the confidence-weighted RMSE to test (ref Mahout's tests)
+ var sqErr = 0.0
+ var denom = 0.0
+ for (u <- 0 until users; p <- 0 until products) {
+ val prediction = predictedRatings.get(u, p)
+ val truePref = truePrefs.get(u, p)
+ val confidence = 1 + 1.0 * trueRatings.get(u, p)
+ val err = confidence * (truePref - prediction) * (truePref - prediction)
+ sqErr += err
+ denom += 1
+ }
+ val rmse = math.sqrt(sqErr / denom)
+ if (math.abs(rmse) > matchThreshold) {
+ fail("Model failed to predict RMSE: %f\ncorr: %s\npred: %s\nU: %s\n P: %s".format(
+ rmse, truePrefs, predictedRatings, predictedU, predictedP))
}
}
}
diff --git a/pom.xml b/pom.xml
index c561b099ab..5ad7b1befb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,11 +21,11 @@
<parent>
<groupId>org.apache</groupId>
<artifactId>apache</artifactId>
- <version>11</version>
+ <version>13</version>
</parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<packaging>pom</packaging>
<name>Spark Project Parent POM</name>
<url>http://spark.incubator.apache.org/</url>
@@ -38,7 +38,9 @@
</licenses>
<scm>
<connection>scm:git:git@github.com:apache/incubator-spark.git</connection>
+ <developerConnection>scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git</developerConnection>
<url>scm:git:git@github.com:apache/incubator-spark.git</url>
+ <tag>HEAD</tag>
</scm>
<developers>
<developer>
@@ -59,6 +61,29 @@
<maven>3.0.0</maven>
</prerequisites>
+ <mailingLists>
+ <mailingList>
+ <name>Dev Mailing List</name>
+ <post>dev@spark.incubator.apache.org</post>
+ <subscribe>dev-subscribe@spark.incubator.apache.org</subscribe>
+ <unsubscribe>dev-unsubscribe@spark.incubator.apache.org</unsubscribe>
+ </mailingList>
+
+ <mailingList>
+ <name>User Mailing List</name>
+ <post>user@spark.incubator.apache.org</post>
+ <subscribe>user-subscribe@spark.incubator.apache.org</subscribe>
+ <unsubscribe>user-unsubscribe@spark.incubator.apache.org</unsubscribe>
+ </mailingList>
+
+ <mailingList>
+ <name>Commits Mailing List</name>
+ <post>commits@spark.incubator.apache.org</post>
+ <subscribe>commits-subscribe@spark.incubator.apache.org</subscribe>
+ <unsubscribe>commits-unsubscribe@spark.incubator.apache.org</unsubscribe>
+ </mailingList>
+ </mailingLists>
+
<modules>
<module>core</module>
<module>bagel</module>
@@ -76,11 +101,12 @@
<java.version>1.5</java.version>
<scala.version>2.9.3</scala.version>
- <mesos.version>0.12.1</mesos.version>
+ <mesos.version>0.13.0</mesos.version>
<akka.version>2.0.5</akka.version>
<slf4j.version>1.7.2</slf4j.version>
<log4j.version>1.2.17</log4j.version>
<hadoop.version>1.0.4</hadoop.version>
+ <yarn.version>0.23.7</yarn.version>
<hbase.version>0.94.6</hbase.version>
<PermGen>64m</PermGen>
@@ -287,6 +313,11 @@
<version>3.0.0</version>
</dependency>
<dependency>
+ <groupId>com.codahale.metrics</groupId>
+ <artifactId>metrics-ganglia</artifactId>
+ <version>3.0.0</version>
+ </dependency>
+ <dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-compiler</artifactId>
<version>${scala.version}</version>
@@ -315,7 +346,7 @@
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<version>1.9.1</version>
<scope>test</scope>
</dependency>
@@ -327,7 +358,7 @@
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.9.3</artifactId>
<version>1.10.0</version>
<scope>test</scope>
</dependency>
@@ -338,6 +369,17 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>org.apache.zookeeper</groupId>
+ <artifactId>zookeeper</artifactId>
+ <version>3.4.5</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-client</artifactId>
<version>${hadoop.version}</version>
@@ -368,6 +410,104 @@
</exclusion>
</exclusions>
</dependency>
+ <dependency>
+ <groupId>net.java.dev.jets3t</groupId>
+ <artifactId>jets3t</artifactId>
+ <version>0.7.1</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ <version>${yarn.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>asm</groupId>
+ <artifactId>asm</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-core-asl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-mapper-asl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-jaxrs</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-xc</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-common</artifactId>
+ <version>${yarn.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>asm</groupId>
+ <artifactId>asm</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-core-asl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-mapper-asl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-jaxrs</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-xc</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-client</artifactId>
+ <version>${yarn.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>asm</groupId>
+ <artifactId>asm</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-core-asl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-mapper-asl</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-jaxrs</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.codehaus.jackson</groupId>
+ <artifactId>jackson-xc</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
<!-- Specify Avro version because Kafka also has it as a dependency -->
<dependency>
<groupId>org.apache.avro</groupId>
@@ -452,7 +592,6 @@
<useZincServer>true</useZincServer>
<args>
<arg>-unchecked</arg>
- <arg>-optimise</arg>
<arg>-deprecation</arg>
</args>
<jvmArgs>
@@ -499,7 +638,7 @@
<junitxml>.</junitxml>
<filereports>${project.build.directory}/SparkTestSuite.txt</filereports>
<argLine>-Xms64m -Xmx3g</argLine>
- <stderr/>
+ <stderr />
</configuration>
<executions>
<execution>
@@ -596,8 +735,8 @@
<properties>
<hadoop.major.version>2</hadoop.major.version>
<!-- 0.23.* is same as 2.0.* - except hardened to run production jobs -->
- <!-- <hadoop.version>0.23.7</hadoop.version> -->
- <hadoop.version>2.0.5-alpha</hadoop.version>
+ <hadoop.version>0.23.7</hadoop.version>
+ <!--<hadoop.version>2.0.5-alpha</hadoop.version> -->
</properties>
<modules>
@@ -620,131 +759,6 @@
<dependencyManagement>
<dependencies>
- <!-- TODO: check versions, bringover from yarn branch ! -->
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-client</artifactId>
- <version>${hadoop.version}</version>
- <exclusions>
- <exclusion>
- <groupId>asm</groupId>
- <artifactId>asm</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.jboss.netty</groupId>
- <artifactId>netty</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-core-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-mapper-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-jaxrs</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-xc</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-api</artifactId>
- <version>${hadoop.version}</version>
- <exclusions>
- <exclusion>
- <groupId>asm</groupId>
- <artifactId>asm</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.jboss.netty</groupId>
- <artifactId>netty</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-core-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-mapper-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-jaxrs</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-xc</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-common</artifactId>
- <version>${hadoop.version}</version>
- <exclusions>
- <exclusion>
- <groupId>asm</groupId>
- <artifactId>asm</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.jboss.netty</groupId>
- <artifactId>netty</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-core-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-mapper-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-jaxrs</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-xc</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-client</artifactId>
- <version>${hadoop.version}</version>
- <exclusions>
- <exclusion>
- <groupId>asm</groupId>
- <artifactId>asm</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.jboss.netty</groupId>
- <artifactId>netty</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-core-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-mapper-asl</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-jaxrs</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.codehaus.jackson</groupId>
- <artifactId>jackson-xc</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
</dependencies>
</dependencyManagement>
</profile>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index bc2cec0553..d7e82079cf 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -33,15 +33,19 @@ object SparkBuild extends Build {
// HBase version; set as appropriate.
val HBASE_VERSION = "0.94.6"
+ // Target JVM version
+ val SCALAC_JVM_VERSION = "jvm-1.5"
+ val JAVAC_JVM_VERSION = "1.5"
+
lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects: _*)
lazy val core = Project("core", file("core"), settings = coreSettings)
lazy val repl = Project("repl", file("repl"), settings = replSettings)
- .dependsOn(core, bagel, mllib) dependsOn(maybeYarn: _*)
+ .dependsOn(core, bagel, mllib)
lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
- .dependsOn(core, mllib, bagel, streaming) dependsOn(maybeYarn: _*)
+ .dependsOn(core, mllib, bagel, streaming)
lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming)
@@ -80,9 +84,11 @@ object SparkBuild extends Build {
def sharedSettings = Defaults.defaultSettings ++ Seq(
organization := "org.apache.spark",
- version := "0.8.0-SNAPSHOT",
+ version := "0.9.0-incubating-SNAPSHOT",
scalaVersion := "2.9.3",
- scalacOptions := Seq("-unchecked", "-optimize", "-deprecation"),
+ scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation",
+ "-target:" + SCALAC_JVM_VERSION),
+ javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION),
unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath },
retrieveManaged := true,
retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]",
@@ -96,6 +102,9 @@ object SparkBuild extends Build {
// Only allow one test at a time, even across projects, since they run in the same JVM
concurrentRestrictions in Global += Tags.limit(Tags.Test, 1),
+ // 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/"),
@@ -152,6 +161,7 @@ object SparkBuild extends Build {
*/
+
libraryDependencies ++= Seq(
"org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106",
"org.scalatest" %% "scalatest" % "1.9.1" % "test",
@@ -174,6 +184,7 @@ object SparkBuild extends Build {
val slf4jVersion = "1.7.2"
+ val excludeCglib = ExclusionRule(organization = "org.sonatype.sisu.inject")
val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson")
val excludeNetty = ExclusionRule(organization = "org.jboss.netty")
val excludeAsm = ExclusionRule(organization = "asm")
@@ -203,15 +214,18 @@ object SparkBuild extends Build {
"it.unimi.dsi" % "fastutil" % "6.4.4",
"colt" % "colt" % "1.2.0",
"net.liftweb" % "lift-json_2.9.2" % "2.5",
- "org.apache.mesos" % "mesos" % "0.12.1",
+ "org.apache.mesos" % "mesos" % "0.13.0",
"io.netty" % "netty-all" % "4.0.0.Beta2",
"org.apache.derby" % "derby" % "10.4.2.0" % "test",
- "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm),
+ "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+ "net.java.dev.jets3t" % "jets3t" % "0.7.1",
"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",
+ "com.codahale.metrics" % "metrics-ganglia" % "3.0.0",
"com.twitter" % "chill_2.9.3" % "0.3.1",
"com.twitter" % "chill-java" % "0.3.1"
)
@@ -242,6 +256,7 @@ object SparkBuild extends Build {
exclude("log4j","log4j")
exclude("org.apache.cassandra.deps", "avro")
excludeAll(excludeSnappy)
+ excludeAll(excludeCglib)
)
) ++ assemblySettings ++ extraAssemblySettings
@@ -284,10 +299,10 @@ object SparkBuild extends Build {
def yarnEnabledSettings = Seq(
libraryDependencies ++= Seq(
// Exclude rule required for all ?
- "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm),
- "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm),
- "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm),
- "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm)
+ "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+ "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+ "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+ "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib)
)
)
@@ -303,6 +318,7 @@ object SparkBuild extends Build {
mergeStrategy in assembly := {
case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
case m if m.toLowerCase.matches("meta-inf.*\\.sf$") => MergeStrategy.discard
+ case "META-INF/services/org.apache.hadoop.fs.FileSystem" => MergeStrategy.concat
case "reference.conf" => MergeStrategy.concat
case _ => MergeStrategy.first
}
diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala
index 999611982a..6a66bd1d06 100644
--- a/project/project/SparkPluginBuild.scala
+++ b/project/project/SparkPluginBuild.scala
@@ -1,7 +1,24 @@
+/*
+ * 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.
+ */
+
import sbt._
object SparkPluginDef extends Build {
lazy val root = Project("plugins", file(".")) dependsOn(junitXmlListener)
/* This is not published in a Maven repository, so we get it from GitHub directly */
lazy val junitXmlListener = uri("git://github.com/ijuma/junit_xml_listener.git#fe434773255b451a38e8d889536ebc260f4225ce")
-} \ No newline at end of file
+}
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index fd5972d381..1f35f6f939 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -30,6 +30,8 @@ Public classes:
An "add-only" shared variable that tasks can only add values to.
- L{SparkFiles<pyspark.files.SparkFiles>}
Access files shipped with jobs.
+ - L{StorageLevel<pyspark.storagelevel.StorageLevel>}
+ Finer-grained cache persistence levels.
"""
import sys
import os
@@ -39,6 +41,7 @@ sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.eg
from pyspark.context import SparkContext
from pyspark.rdd import RDD
from pyspark.files import SparkFiles
+from pyspark.storagelevel import StorageLevel
-__all__ = ["SparkContext", "RDD", "SparkFiles"]
+__all__ = ["SparkContext", "RDD", "SparkFiles", "StorageLevel"]
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 8fbf296509..597110321a 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -27,6 +27,7 @@ from pyspark.broadcast import Broadcast
from pyspark.files import SparkFiles
from pyspark.java_gateway import launch_gateway
from pyspark.serializers import dump_pickle, write_with_length, batched
+from pyspark.storagelevel import StorageLevel
from pyspark.rdd import RDD
from py4j.java_collections import ListConverter
@@ -279,6 +280,16 @@ class SparkContext(object):
"""
self._jsc.sc().setCheckpointDir(dirName, useExisting)
+ def _getJavaStorageLevel(self, storageLevel):
+ """
+ Returns a Java StorageLevel based on a pyspark.StorageLevel.
+ """
+ if not isinstance(storageLevel, StorageLevel):
+ raise Exception("storageLevel must be of type pyspark.StorageLevel")
+
+ newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel
+ return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory,
+ storageLevel.deserialized, storageLevel.replication)
def _test():
import atexit
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 914118ccdd..7019fb8bee 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -29,7 +29,7 @@ from threading import Thread
from pyspark import cloudpickle
from pyspark.serializers import batched, Batch, dump_pickle, load_pickle, \
- read_from_pickle_file
+ read_from_pickle_file, pack_long
from pyspark.join import python_join, python_left_outer_join, \
python_right_outer_join, python_cogroup
from pyspark.statcounter import StatCounter
@@ -70,6 +70,25 @@ class RDD(object):
self._jrdd.cache()
return self
+ def persist(self, storageLevel):
+ """
+ Set this RDD's storage level to persist its values across operations after the first time
+ it is computed. This can only be used to assign a new storage level if the RDD does not
+ have a storage level set yet.
+ """
+ self.is_cached = True
+ javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel)
+ self._jrdd.persist(javaStorageLevel)
+ return self
+
+ def unpersist(self):
+ """
+ Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+ """
+ self.is_cached = False
+ self._jrdd.unpersist()
+ return self
+
def checkpoint(self):
"""
Mark this RDD for checkpointing. It will be saved to a file inside the
@@ -98,8 +117,6 @@ class RDD(object):
else:
return None
- # TODO persist(self, storageLevel)
-
def map(self, f, preservesPartitioning=False):
"""
Return a new RDD containing the distinct elements in this RDD.
@@ -208,7 +225,7 @@ class RDD(object):
total = num
samples = self.sample(withReplacement, fraction, seed).collect()
-
+
# If the first sample didn't turn out large enough, keep trying to take samples;
# this shouldn't happen often because we use a big multiplier for their initial size.
# See: scala/spark/RDD.scala
@@ -244,7 +261,55 @@ class RDD(object):
raise TypeError
return self.union(other)
- # TODO: sort
+ def sortByKey(self, ascending=True, numPartitions=None, keyfunc = lambda x: x):
+ """
+ Sorts this RDD, which is assumed to consist of (key, value) pairs.
+
+ >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)]
+ >>> sc.parallelize(tmp).sortByKey(True, 2).collect()
+ [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)]
+ >>> tmp2 = [('Mary', 1), ('had', 2), ('a', 3), ('little', 4), ('lamb', 5)]
+ >>> tmp2.extend([('whose', 6), ('fleece', 7), ('was', 8), ('white', 9)])
+ >>> sc.parallelize(tmp2).sortByKey(True, 3, keyfunc=lambda k: k.lower()).collect()
+ [('a', 3), ('fleece', 7), ('had', 2), ('lamb', 5), ('little', 4), ('Mary', 1), ('was', 8), ('white', 9), ('whose', 6)]
+ """
+ if numPartitions is None:
+ numPartitions = self.ctx.defaultParallelism
+
+ bounds = list()
+
+ # first compute the boundary of each part via sampling: we want to partition
+ # the key-space into bins such that the bins have roughly the same
+ # number of (key, value) pairs falling into them
+ if numPartitions > 1:
+ rddSize = self.count()
+ maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner
+ fraction = min(maxSampleSize / max(rddSize, 1), 1.0)
+
+ samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect()
+ samples = sorted(samples, reverse=(not ascending), key=keyfunc)
+
+ # we have numPartitions many parts but one of the them has
+ # an implicit boundary
+ for i in range(0, numPartitions - 1):
+ index = (len(samples) - 1) * (i + 1) / numPartitions
+ bounds.append(samples[index])
+
+ def rangePartitionFunc(k):
+ p = 0
+ while p < len(bounds) and keyfunc(k) > bounds[p]:
+ p += 1
+ if ascending:
+ return p
+ else:
+ return numPartitions-1-p
+
+ def mapFunc(iterator):
+ yield sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k))
+
+ return (self.partitionBy(numPartitions, partitionFunc=rangePartitionFunc)
+ .mapPartitions(mapFunc,preservesPartitioning=True)
+ .flatMap(lambda x: x, preservesPartitioning=True))
def glom(self):
"""
@@ -406,7 +471,7 @@ class RDD(object):
3
"""
return self.mapPartitions(lambda i: [sum(1 for _ in i)]).sum()
-
+
def stats(self):
"""
Return a L{StatCounter} object that captures the mean, variance
@@ -443,7 +508,7 @@ class RDD(object):
0.816...
"""
return self.stats().stdev()
-
+
def sampleStdev(self):
"""
Compute the sample standard deviation of this RDD's elements (which corrects for bias in
@@ -671,11 +736,13 @@ class RDD(object):
# form the hash buckets in Python, transferring O(numPartitions) objects
# to Java. Each object is a (splitNumber, [objects]) pair.
def add_shuffle_key(split, iterator):
+
buckets = defaultdict(list)
+
for (k, v) in iterator:
buckets[partitionFunc(k) % numPartitions].append((k, v))
for (split, items) in buckets.iteritems():
- yield str(split)
+ yield pack_long(split)
yield dump_pickle(Batch(items))
keyed = PipelinedRDD(self, add_shuffle_key)
keyed._bypass_serializer = True
@@ -811,9 +878,9 @@ class RDD(object):
>>> y = sc.parallelize([("a", 3), ("c", None)])
>>> sorted(x.subtractByKey(y).collect())
[('b', 4), ('b', 5)]
- """
- filter_func = lambda tpl: len(tpl[1][0]) > 0 and len(tpl[1][1]) == 0
- map_func = lambda tpl: [(tpl[0], val) for val in tpl[1][0]]
+ """
+ filter_func = lambda (key, vals): len(vals[0]) > 0 and len(vals[1]) == 0
+ map_func = lambda (key, vals): [(key, val) for val in vals[0]]
return self.cogroup(other, numPartitions).filter(filter_func).flatMap(map_func)
def subtract(self, other, numPartitions=None):
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index fecacd1241..54fed1c9c7 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -67,6 +67,10 @@ def write_long(value, stream):
stream.write(struct.pack("!q", value))
+def pack_long(value):
+ return struct.pack("!q", value)
+
+
def read_int(stream):
length = stream.read(4)
if length == "":
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index 54823f8037..a475959090 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -24,6 +24,7 @@ import os
import platform
import pyspark
from pyspark.context import SparkContext
+from pyspark.storagelevel import StorageLevel
# this is the equivalent of ADD_JARS
add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None
@@ -34,7 +35,7 @@ print """Welcome to
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
- /__ / .__/\_,_/_/ /_/\_\ version 0.8.0
+ /__ / .__/\_,_/_/ /_/\_\ version 0.9.0-SNAPSHOT
/_/
"""
print "Using Python version %s (%s, %s)" % (
diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py
new file mode 100644
index 0000000000..b31f4762e6
--- /dev/null
+++ b/python/pyspark/storagelevel.py
@@ -0,0 +1,43 @@
+#
+# 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.
+#
+
+__all__ = ["StorageLevel"]
+
+class StorageLevel:
+ """
+ Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory,
+ whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory
+ in a serialized format, and whether to replicate the RDD partitions on multiple nodes.
+ Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY.
+ """
+
+ def __init__(self, useDisk, useMemory, deserialized, replication = 1):
+ self.useDisk = useDisk
+ self.useMemory = useMemory
+ self.deserialized = deserialized
+ self.replication = replication
+
+StorageLevel.DISK_ONLY = StorageLevel(True, False, False)
+StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, 2)
+StorageLevel.MEMORY_ONLY = StorageLevel(False, True, True)
+StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, True, 2)
+StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False)
+StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, 2)
+StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, True)
+StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, True, 2)
+StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False)
+StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, 2)
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index d61b36a61a..f6bf94be6b 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -21,12 +21,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl-bin</artifactId>
+ <artifactId>spark-repl-bin_2.9.3</artifactId>
<packaging>pom</packaging>
<name>Spark Project REPL binary packaging</name>
<url>http://spark.incubator.apache.org/</url>
@@ -40,18 +40,18 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel</artifactId>
+ <artifactId>spark-bagel_2.9.3</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl</artifactId>
+ <artifactId>spark-repl_2.9.3</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
@@ -89,7 +89,7 @@
</goals>
<configuration>
<transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>reference.conf</resource>
</transformer>
@@ -106,16 +106,6 @@
<profiles>
<profile>
- <id>hadoop2-yarn</id>
- <dependencies>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn</artifactId>
- <version>${project.version}</version>
- </dependency>
- </dependencies>
- </profile>
- <profile>
<id>deb</id>
<build>
<plugins>
diff --git a/repl/pom.xml b/repl/pom.xml
index a1c87d7618..49d86621dd 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -21,12 +21,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl</artifactId>
+ <artifactId>spark-repl_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project REPL</name>
<url>http://spark.incubator.apache.org/</url>
@@ -39,18 +39,18 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel</artifactId>
+ <artifactId>spark-bagel_2.9.3</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib</artifactId>
+ <artifactId>spark-mllib_2.9.3</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
@@ -76,12 +76,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.9.3</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
@@ -101,14 +101,14 @@
<configuration>
<exportAntProperties>true</exportAntProperties>
<tasks>
- <property name="spark.classpath" refid="maven.test.classpath"/>
- <property environment="env"/>
+ <property name="spark.classpath" refid="maven.test.classpath" />
+ <property environment="env" />
<fail message="Please set the SCALA_HOME (or SCALA_LIBRARY_PATH if scala is on the path) environment variables and retry.">
<condition>
<not>
<or>
- <isset property="env.SCALA_HOME"/>
- <isset property="env.SCALA_LIBRARY_PATH"/>
+ <isset property="env.SCALA_HOME" />
+ <isset property="env.SCALA_LIBRARY_PATH" />
</or>
</not>
</condition>
@@ -131,16 +131,4 @@
</plugin>
</plugins>
</build>
- <profiles>
- <profile>
- <id>hadoop2-yarn</id>
- <dependencies>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn</artifactId>
- <version>${project.version}</version>
- </dependency>
- </dependencies>
- </profile>
- </profiles>
</project>
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 193ccb48ee..36f54a22cf 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -200,7 +200,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
- /___/ .__/\_,_/_/ /_/\_\ version 0.8.0
+ /___/ .__/\_,_/_/ /_/\_\ version 0.9.0-SNAPSHOT
/_/
""")
import Properties._
diff --git a/run-example b/run-example
index 24d83ba5cf..08ec717ca5 100755
--- a/run-example
+++ b/run-example
@@ -39,14 +39,14 @@ fi
# to avoid the -sources and -doc packages that are built by publish-local.
EXAMPLES_DIR="$FWDIR"/examples
SPARK_EXAMPLES_JAR=""
-if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9T].jar ]; then
+if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then
# Use the JAR from the SBT build
- export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9T].jar`
+ export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar`
fi
-if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9T].jar ]; then
+if [ -e "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar ]; then
# Use the JAR from the Maven build
# TODO: this also needs to become an assembly!
- export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9T].jar`
+ export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/spark-examples*[0-9Tg].jar`
fi
if [[ -z $SPARK_EXAMPLES_JAR ]]; then
echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
diff --git a/spark-class b/spark-class
index 037abda3b7..e111ef6da7 100755
--- a/spark-class
+++ b/spark-class
@@ -37,7 +37,7 @@ fi
# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
# values for that; it doesn't need a lot
-if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then
+if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then
SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
# Do not overwrite SPARK_JAVA_OPTS environment variable in this script
@@ -49,19 +49,19 @@ fi
# Add java opts for master, worker, executor. The opts maybe null
case "$1" in
- 'spark.deploy.master.Master')
+ 'org.apache.spark.deploy.master.Master')
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
;;
- 'spark.deploy.worker.Worker')
+ 'org.apache.spark.deploy.worker.Worker')
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
;;
- 'spark.executor.StandaloneExecutorBackend')
+ 'org.apache.spark.executor.StandaloneExecutorBackend')
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
;;
- 'spark.executor.MesosExecutorBackend')
+ 'org.apache.spark.executor.MesosExecutorBackend')
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
;;
- 'spark.repl.Main')
+ 'org.apache.spark.repl.Main')
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
;;
esac
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 7bea069b61..3b25fb49fb 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -21,12 +21,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming</artifactId>
+ <artifactId>spark-streaming_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project Streaming</name>
<url>http://spark.incubator.apache.org/</url>
@@ -42,7 +42,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -58,6 +58,7 @@
<groupId>org.apache.kafka</groupId>
<artifactId>kafka</artifactId>
<version>0.7.2-spark</version> <!-- Comes from our in-project repository -->
+ <scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.flume</groupId>
@@ -91,12 +92,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_${scala.version}</artifactId>
+ <artifactId>scalacheck_2.9.3</artifactId>
<scope>test</scope>
</dependency>
<dependency>
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 aae79a4e6f..b97fb7e6e3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
@@ -30,10 +30,11 @@ import akka.actor._
import akka.pattern.ask
import akka.util.duration._
import akka.dispatch._
+import org.apache.spark.storage.BlockId
private[streaming] sealed trait NetworkInputTrackerMessage
private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage
-private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[String], metadata: Any) extends NetworkInputTrackerMessage
+private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) extends NetworkInputTrackerMessage
private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage
/**
@@ -48,7 +49,7 @@ class NetworkInputTracker(
val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*)
val receiverExecutor = new ReceiverExecutor()
val receiverInfo = new HashMap[Int, ActorRef]
- val receivedBlockIds = new HashMap[Int, Queue[String]]
+ val receivedBlockIds = new HashMap[Int, Queue[BlockId]]
val timeout = 5000.milliseconds
var currentTime: Time = null
@@ -67,9 +68,9 @@ class NetworkInputTracker(
}
/** Return all the blocks received from a receiver. */
- def getBlockIds(receiverId: Int, time: Time): Array[String] = synchronized {
+ def getBlockIds(receiverId: Int, time: Time): Array[BlockId] = synchronized {
val queue = receivedBlockIds.synchronized {
- receivedBlockIds.getOrElse(receiverId, new Queue[String]())
+ receivedBlockIds.getOrElse(receiverId, new Queue[BlockId]())
}
val result = queue.synchronized {
queue.dequeueAll(x => true)
@@ -92,7 +93,7 @@ class NetworkInputTracker(
case AddBlocks(streamId, blockIds, metadata) => {
val tmp = receivedBlockIds.synchronized {
if (!receivedBlockIds.contains(streamId)) {
- receivedBlockIds += ((streamId, new Queue[String]))
+ receivedBlockIds += ((streamId, new Queue[BlockId]))
}
receivedBlockIds(streamId)
}
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 31f9891560..8d3ac0fc65 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
@@ -31,7 +31,7 @@ import org.apache.spark.streaming.util.{RecurringTimer, SystemClock}
import org.apache.spark.streaming._
import org.apache.spark.{Logging, SparkEnv}
import org.apache.spark.rdd.{RDD, BlockRDD}
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId}
/**
* Abstract class for defining any InputDStream that has to start a receiver on worker
@@ -69,7 +69,7 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming
val blockIds = ssc.networkInputTracker.getBlockIds(id, validTime)
Some(new BlockRDD[T](ssc.sc, blockIds))
} else {
- Some(new BlockRDD[T](ssc.sc, Array[String]()))
+ Some(new BlockRDD[T](ssc.sc, Array[BlockId]()))
}
}
}
@@ -77,7 +77,7 @@ abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : Streaming
private[streaming] sealed trait NetworkReceiverMessage
private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage
-private[streaming] case class ReportBlock(blockId: String, metadata: Any) extends NetworkReceiverMessage
+private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) extends NetworkReceiverMessage
private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage
/**
@@ -158,7 +158,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
/**
* Pushes a block (as an ArrayBuffer filled with data) into the block manager.
*/
- def pushBlock(blockId: String, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) {
+ def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) {
env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level)
actor ! ReportBlock(blockId, metadata)
}
@@ -166,7 +166,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
/**
* Pushes a block (as bytes) into the block manager.
*/
- def pushBlock(blockId: String, bytes: ByteBuffer, metadata: Any, level: StorageLevel) {
+ def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) {
env.blockManager.putBytes(blockId, bytes, level)
actor ! ReportBlock(blockId, metadata)
}
@@ -209,7 +209,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
class BlockGenerator(storageLevel: StorageLevel)
extends Serializable with Logging {
- case class Block(id: String, buffer: ArrayBuffer[T], metadata: Any = null)
+ case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null)
val clock = new SystemClock()
val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong
@@ -241,7 +241,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
val newBlockBuffer = currentBuffer
currentBuffer = new ArrayBuffer[T]
if (newBlockBuffer.size > 0) {
- val blockId = "input-" + NetworkReceiver.this.streamId + "-" + (time - blockInterval)
+ val blockId = StreamBlockId(NetworkReceiver.this.streamId, time - blockInterval)
val newBlock = new Block(blockId, newBlockBuffer)
blocksForPushing.add(newBlock)
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
index c91f12ecd7..10ed4ef78d 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
@@ -18,7 +18,7 @@
package org.apache.spark.streaming.dstream
import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{StorageLevel, StreamBlockId}
import org.apache.spark.streaming.StreamingContext
import java.net.InetSocketAddress
@@ -71,7 +71,7 @@ class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel)
var nextBlockNumber = 0
while (true) {
val buffer = queue.take()
- val blockId = "input-" + streamId + "-" + nextBlockNumber
+ val blockId = StreamBlockId(streamId, nextBlockNumber)
nextBlockNumber += 1
pushBlock(blockId, buffer, null, storageLevel)
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
index 4b5d8c467e..ef0f85a717 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala
@@ -21,7 +21,7 @@ import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy }
import akka.actor.{ actorRef2Scala, ActorRef }
import akka.actor.{ PossiblyHarmful, OneForOneStrategy }
-import org.apache.spark.storage.StorageLevel
+import org.apache.spark.storage.{StorageLevel, StreamBlockId}
import org.apache.spark.streaming.dstream.NetworkReceiver
import java.util.concurrent.atomic.AtomicInteger
@@ -159,7 +159,7 @@ private[streaming] class ActorReceiver[T: ClassManifest](
protected def pushBlock(iter: Iterator[T]) {
val buffer = new ArrayBuffer[T]
buffer ++= iter
- pushBlock("block-" + streamId + "-" + System.nanoTime(), buffer, null, storageLevel)
+ pushBlock(StreamBlockId(streamId, System.nanoTime()), buffer, null, storageLevel)
}
protected def onStart() = {
diff --git a/tools/pom.xml b/tools/pom.xml
index 77646a6816..f1c489beea 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -20,12 +20,12 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-tools</artifactId>
+ <artifactId>spark-tools_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project Tools</name>
<url>http://spark.incubator.apache.org/</url>
@@ -33,17 +33,17 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
+ <artifactId>spark-core_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming</artifactId>
+ <artifactId>spark-streaming_2.9.3</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_${scala.version}</artifactId>
+ <artifactId>scalatest_2.9.3</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 654b5bcd2d..3bc619df07 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -20,16 +20,49 @@
<parent>
<groupId>org.apache.spark</groupId>
<artifactId>spark-parent</artifactId>
- <version>0.8.0-SNAPSHOT</version>
+ <version>0.9.0-incubating-SNAPSHOT</version>
<relativePath>../pom.xml</relativePath>
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn</artifactId>
+ <artifactId>spark-yarn_2.9.3</artifactId>
<packaging>jar</packaging>
<name>Spark Project YARN Support</name>
<url>http://spark.incubator.apache.org/</url>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-core_2.9.3</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-client</artifactId>
+ <version>${yarn.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <artifactId>avro</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <artifactId>avro-ipc</artifactId>
+ </dependency>
+ </dependencies>
+
<build>
<outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
@@ -64,7 +97,7 @@
</goals>
<configuration>
<transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>reference.conf</resource>
</transformer>
@@ -75,37 +108,4 @@
</plugin>
</plugins>
</build>
-
- <profiles>
- <profile>
- <id>hadoop2-yarn</id>
- <dependencies>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-core</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-api</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-common</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-yarn-client</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.avro</groupId>
- <artifactId>avro</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.avro</groupId>
- <artifactId>avro-ipc</artifactId>
- </dependency>
- </dependencies>
- </profile>
- </profiles>
</project>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 139a977a03..c1a87d3373 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -17,21 +17,25 @@
package org.apache.spark.deploy.yarn
+import java.io.IOException;
import java.net.Socket
+import java.security.PrivilegedExceptionAction
import java.util.concurrent.CopyOnWriteArrayList
import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.util.ShutdownHookManager
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.api.protocolrecords._
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
-import scala.collection.JavaConversions._
-import org.apache.spark.{SparkContext, Logging, Utils}
+import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.util.Utils
import org.apache.hadoop.security.UserGroupInformation
-import java.security.PrivilegedExceptionAction
+import scala.collection.JavaConversions._
class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
@@ -42,38 +46,46 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
private var appAttemptId: ApplicationAttemptId = null
private var userThread: Thread = null
private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+ private val fs = FileSystem.get(yarnConf)
private var yarnAllocator: YarnAllocationHandler = null
private var isFinished:Boolean = false
+ private var uiAddress: String = ""
+ private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES,
+ YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES)
+ private var isLastAMRetry: Boolean = true
+
def run() {
// setup the directories so things go to yarn approved directories rather
// then user specified and /tmp
System.setProperty("spark.local.dir", getLocalDirs())
+
+ // use priority 30 as its higher then HDFS. Its same priority as MapReduce is using
+ ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
appAttemptId = getApplicationAttemptId()
+ isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts;
resourceManager = registerWithResourceManager()
- val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
- // Compute number of threads for akka
- val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory()
+ // Workaround until hadoop moves to something which has
+ // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line)
+ // ignore result
+ // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times
+ // Hence args.workerCores = numCore disabled above. Any better option ?
- if (minimumMemory > 0) {
- val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD
- val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0)
+ // Compute number of threads for akka
+ //val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory()
+ //if (minimumMemory > 0) {
+ // val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD
+ // val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0)
- if (numCore > 0) {
+ // if (numCore > 0) {
// do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406
// TODO: Uncomment when hadoop is on a version which has this fixed.
// args.workerCores = numCore
- }
- }
-
- // Workaround until hadoop moves to something which has
- // https://issues.apache.org/jira/browse/HADOOP-8406
- // ignore result
- // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times
- // Hence args.workerCores = numCore disabled above. Any better option ?
+ // }
+ //}
// org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
ApplicationMaster.register(this)
@@ -83,6 +95,11 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
// This a bit hacky, but we need to wait until the spark.driver.port property has
// been set by the Thread executing the user class.
waitForSparkMaster()
+
+ waitForSparkContextInitialized()
+
+ // do this after spark master is up and SparkContext is created so that we can register UI Url
+ val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
// Allocate all containers
allocateWorkers()
@@ -134,8 +151,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
// Users can then monitor stderr/stdout on that node if required.
appMasterRequest.setHost(Utils.localHostName())
appMasterRequest.setRpcPort(0)
- // What do we provide here ? Might make sense to expose something sensible later ?
- appMasterRequest.setTrackingUrl("")
+ appMasterRequest.setTrackingUrl(uiAddress)
return resourceManager.registerApplicationMaster(appMasterRequest)
}
@@ -143,7 +159,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
logInfo("Waiting for spark driver to be reachable.")
var driverUp = false
var tries = 0
- while(!driverUp && tries < 10) {
+ val numTries = System.getProperty("spark.yarn.applicationMaster.waitTries", "10").toInt
+ while(!driverUp && tries < numTries) {
val driverHost = System.getProperty("spark.driver.host")
val driverPort = System.getProperty("spark.driver.port")
try {
@@ -177,6 +194,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
// It need shutdown hook to set SUCCEEDED
successed = true
} finally {
+ logDebug("finishing main")
+ isLastAMRetry = true;
if (successed) {
ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
} else {
@@ -189,24 +208,42 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
return t
}
- private def allocateWorkers() {
+ // this need to happen before allocateWorkers
+ private def waitForSparkContextInitialized() {
logInfo("Waiting for spark context initialization")
-
try {
var sparkContext: SparkContext = null
ApplicationMaster.sparkContextRef.synchronized {
var count = 0
- while (ApplicationMaster.sparkContextRef.get() == null && count < 10) {
+ val waitTime = 10000L
+ val numTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt
+ while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) {
logInfo("Waiting for spark context initialization ... " + count)
count = count + 1
- ApplicationMaster.sparkContextRef.wait(10000L)
+ ApplicationMaster.sparkContextRef.wait(waitTime)
}
sparkContext = ApplicationMaster.sparkContextRef.get()
- assert(sparkContext != null)
- this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, sparkContext.preferredNodeLocationData)
+ assert(sparkContext != null || count >= numTries)
+
+ if (null != sparkContext) {
+ uiAddress = sparkContext.ui.appUIAddress
+ this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args,
+ sparkContext.preferredNodeLocationData)
+ } else {
+ logWarning("Unable to retrieve sparkContext inspite of waiting for " + count * waitTime +
+ ", numTries = " + numTries)
+ this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args)
+ }
}
+ } finally {
+ // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT :
+ // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks
+ ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
+ }
+ }
-
+ private def allocateWorkers() {
+ try {
logInfo("Allocating " + args.numWorkers + " workers.")
// Wait until all containers have finished
// TODO: This is a bit ugly. Can we make it nicer?
@@ -298,9 +335,45 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
.asInstanceOf[FinishApplicationMasterRequest]
finishReq.setAppAttemptId(appAttemptId)
finishReq.setFinishApplicationStatus(status)
+ // set tracking url to empty since we don't have a history server
+ finishReq.setTrackingUrl("")
resourceManager.finishApplicationMaster(finishReq)
}
+
+ /**
+ * clean up the staging directory.
+ */
+ private def cleanupStagingDir() {
+ var stagingDirPath: Path = null
+ try {
+ val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+ if (!preserveFiles) {
+ stagingDirPath = new Path(System.getenv("SPARK_YARN_JAR_PATH")).getParent()
+ if (stagingDirPath == null) {
+ logError("Staging directory is null")
+ return
+ }
+ logInfo("Deleting staging directory " + stagingDirPath)
+ fs.delete(stagingDirPath, true)
+ }
+ } catch {
+ case e: IOException =>
+ logError("Failed to cleanup staging dir " + stagingDirPath, e)
+ }
+ }
+
+ // The shutdown hook that runs when a signal is received AND during normal
+ // close of the JVM.
+ class AppMasterShutdownHook(appMaster: ApplicationMaster) extends Runnable {
+
+ def run() {
+ logInfo("AppMaster received a signal.")
+ // we need to clean up staging dir before HDFS is shut down
+ // make sure we don't delete it until this is the last AM
+ if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
+ }
+ }
}
@@ -340,6 +413,8 @@ object ApplicationMaster {
// Add a shutdown hook - as a best case effort in case users do not call sc.stop or do System.exit
// Should not really have to do this, but it helps yarn to evict resources earlier.
// not to mention, prevent Client declaring failure even though we exit'ed properly.
+ // Note that this will unfortunately not properly clean up the staging files because it gets called to
+ // late and the filesystem is already shutdown.
if (modified) {
Runtime.getRuntime().addShutdownHook(new Thread with Logging {
// This is not just to log, but also to ensure that log system is initialized for this instance when we actually are 'run'
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
index f47e23b63f..f76a5ddd39 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
@@ -80,7 +80,7 @@ class ApplicationMasterArguments(val args: Array[String]) {
System.err.println("Unknown/unsupported param " + unknownParam)
}
System.err.println(
- "Usage: spark.deploy.yarn.ApplicationMaster [options] \n" +
+ "Usage: org.apache.spark.deploy.yarn.ApplicationMaster [options] \n" +
"Options:\n" +
" --jar JAR_PATH Path to your application's JAR file (required)\n" +
" --class CLASS_NAME Name of your application's main class (required)\n" +
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 48e737ed79..8afb3e39cb 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -33,7 +33,8 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC
import scala.collection.mutable.HashMap
import scala.collection.JavaConversions._
-import org.apache.spark.{Logging, Utils}
+import org.apache.spark.Logging
+import org.apache.spark.util.Utils
import org.apache.hadoop.yarn.util.{Apps, Records, ConverterUtils}
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.spark.deploy.SparkHadoopUtil
@@ -44,7 +45,13 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
var rpc: YarnRPC = YarnRPC.create(conf)
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
- val credentials = UserGroupInformation.getCurrentUser().getCredentials();
+ val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+ private var distFiles = None: Option[String]
+ private var distFilesTimeStamps = None: Option[String]
+ private var distFilesFileSizes = None: Option[String]
+ private var distArchives = None: Option[String]
+ private var distArchivesTimeStamps = None: Option[String]
+ private var distArchivesFileSizes = None: Option[String]
def run() {
init(yarnConf)
@@ -56,7 +63,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
verifyClusterResources(newApp)
val appContext = createApplicationSubmissionContext(appId)
- val localResources = prepareLocalResources(appId, "spark")
+ val localResources = prepareLocalResources(appId, ".sparkStaging")
val env = setupLaunchEnv(localResources)
val amContainer = createContainerLaunchContext(newApp, localResources, env)
@@ -105,13 +112,76 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
logInfo("Setting up application submission context for ASM")
val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
appContext.setApplicationId(appId)
- appContext.setApplicationName("Spark")
+ appContext.setApplicationName(args.appName)
return appContext
}
-
- def prepareLocalResources(appId: ApplicationId, appName: String): HashMap[String, LocalResource] = {
+
+ /**
+ * Copy the local file into HDFS and configure to be distributed with the
+ * job via the distributed cache.
+ * If a fragment is specified the file will be referenced as that fragment.
+ */
+ private def copyLocalFile(
+ dstDir: Path,
+ resourceType: LocalResourceType,
+ originalPath: Path,
+ replication: Short,
+ localResources: HashMap[String,LocalResource],
+ fragment: String,
+ appMasterOnly: Boolean = false): Unit = {
+ val fs = FileSystem.get(conf)
+ val newPath = new Path(dstDir, originalPath.getName())
+ logInfo("Uploading " + originalPath + " to " + newPath)
+ fs.copyFromLocalFile(false, true, originalPath, newPath)
+ fs.setReplication(newPath, replication);
+ val destStatus = fs.getFileStatus(newPath)
+
+ val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+ amJarRsrc.setType(resourceType)
+ amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
+ amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(newPath))
+ amJarRsrc.setTimestamp(destStatus.getModificationTime())
+ amJarRsrc.setSize(destStatus.getLen())
+ var pathURI: URI = new URI(newPath.toString() + "#" + originalPath.getName());
+ if ((fragment == null) || (fragment.isEmpty())){
+ localResources(originalPath.getName()) = amJarRsrc
+ } else {
+ localResources(fragment) = amJarRsrc
+ pathURI = new URI(newPath.toString() + "#" + fragment);
+ }
+ val distPath = pathURI.toString()
+ if (appMasterOnly == true) return
+ if (resourceType == LocalResourceType.FILE) {
+ distFiles match {
+ case Some(path) =>
+ distFilesFileSizes = Some(distFilesFileSizes.get + "," +
+ destStatus.getLen().toString())
+ distFilesTimeStamps = Some(distFilesTimeStamps.get + "," +
+ destStatus.getModificationTime().toString())
+ distFiles = Some(path + "," + distPath)
+ case _ =>
+ distFilesFileSizes = Some(destStatus.getLen().toString())
+ distFilesTimeStamps = Some(destStatus.getModificationTime().toString())
+ distFiles = Some(distPath)
+ }
+ } else {
+ distArchives match {
+ case Some(path) =>
+ distArchivesTimeStamps = Some(distArchivesTimeStamps.get + "," +
+ destStatus.getModificationTime().toString())
+ distArchivesFileSizes = Some(distArchivesFileSizes.get + "," +
+ destStatus.getLen().toString())
+ distArchives = Some(path + "," + distPath)
+ case _ =>
+ distArchivesTimeStamps = Some(destStatus.getModificationTime().toString())
+ distArchivesFileSizes = Some(destStatus.getLen().toString())
+ distArchives = Some(distPath)
+ }
+ }
+ }
+
+ def prepareLocalResources(appId: ApplicationId, sparkStagingDir: String): HashMap[String, LocalResource] = {
logInfo("Preparing Local resources")
- val locaResources = HashMap[String, LocalResource]()
// Upload Spark and the application JAR to the remote file system
// Add them as local resources to the AM
val fs = FileSystem.get(conf)
@@ -124,33 +194,69 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
}
}
+ val pathSuffix = sparkStagingDir + "/" + appId.toString() + "/"
+ val dst = new Path(fs.getHomeDirectory(), pathSuffix)
+ val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
+
+ if (UserGroupInformation.isSecurityEnabled()) {
+ val dstFs = dst.getFileSystem(conf)
+ dstFs.addDelegationTokens(delegTokenRenewer, credentials);
+ }
+ val localResources = HashMap[String, LocalResource]()
+
Map("spark.jar" -> System.getenv("SPARK_JAR"), "app.jar" -> args.userJar, "log4j.properties" -> System.getenv("SPARK_LOG4J_CONF"))
.foreach { case(destName, _localPath) =>
val localPath: String = if (_localPath != null) _localPath.trim() else ""
if (! localPath.isEmpty()) {
val src = new Path(localPath)
- val pathSuffix = appName + "/" + appId.getId() + destName
- val dst = new Path(fs.getHomeDirectory(), pathSuffix)
- logInfo("Uploading " + src + " to " + dst)
- fs.copyFromLocalFile(false, true, src, dst)
- val destStatus = fs.getFileStatus(dst)
-
- // get tokens for anything we upload to hdfs
- if (UserGroupInformation.isSecurityEnabled()) {
- fs.addDelegationTokens(delegTokenRenewer, credentials);
- }
+ val newPath = new Path(dst, destName)
+ logInfo("Uploading " + src + " to " + newPath)
+ fs.copyFromLocalFile(false, true, src, newPath)
+ fs.setReplication(newPath, replication);
+ val destStatus = fs.getFileStatus(newPath)
val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
amJarRsrc.setType(LocalResourceType.FILE)
amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
- amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(dst))
+ amJarRsrc.setResource(ConverterUtils.getYarnUrlFromPath(newPath))
amJarRsrc.setTimestamp(destStatus.getModificationTime())
amJarRsrc.setSize(destStatus.getLen())
- locaResources(destName) = amJarRsrc
+ localResources(destName) = amJarRsrc
}
}
+
+ // handle any add jars
+ if ((args.addJars != null) && (!args.addJars.isEmpty())){
+ args.addJars.split(',').foreach { case file: String =>
+ val tmpURI = new URI(file)
+ val tmp = new Path(tmpURI)
+ copyLocalFile(dst, LocalResourceType.FILE, tmp, replication, localResources,
+ tmpURI.getFragment(), true)
+ }
+ }
+
+ // handle any distributed cache files
+ if ((args.files != null) && (!args.files.isEmpty())){
+ args.files.split(',').foreach { case file: String =>
+ val tmpURI = new URI(file)
+ val tmp = new Path(tmpURI)
+ copyLocalFile(dst, LocalResourceType.FILE, tmp, replication, localResources,
+ tmpURI.getFragment())
+ }
+ }
+
+ // handle any distributed cache archives
+ if ((args.archives != null) && (!args.archives.isEmpty())) {
+ args.archives.split(',').foreach { case file:String =>
+ val tmpURI = new URI(file)
+ val tmp = new Path(tmpURI)
+ copyLocalFile(dst, LocalResourceType.ARCHIVE, tmp, replication,
+ localResources, tmpURI.getFragment())
+ }
+ }
+
UserGroupInformation.getCurrentUser().addCredentials(credentials);
- return locaResources
+ return localResources
}
def setupLaunchEnv(localResources: HashMap[String, LocalResource]): HashMap[String, String] = {
@@ -159,11 +265,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
val env = new HashMap[String, String]()
- // If log4j present, ensure ours overrides all others
- if (log4jConfLocalRes != null) Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./")
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name,
+ Environment.PWD.$() + Path.SEPARATOR + "*")
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*")
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH")
Client.populateHadoopClasspath(yarnConf, env)
env("SPARK_YARN_MODE") = "true"
env("SPARK_YARN_JAR_PATH") =
@@ -185,6 +290,18 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
env("SPARK_YARN_LOG4J_SIZE") = log4jConfLocalRes.getSize().toString()
}
+ // set the environment variables to be passed on to the Workers
+ if (distFiles != None) {
+ env("SPARK_YARN_CACHE_FILES") = distFiles.get
+ env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = distFilesTimeStamps.get
+ env("SPARK_YARN_CACHE_FILES_FILE_SIZES") = distFilesFileSizes.get
+ }
+ if (distArchives != None) {
+ env("SPARK_YARN_CACHE_ARCHIVES") = distArchives.get
+ env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = distArchivesTimeStamps.get
+ env("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") = distArchivesFileSizes.get
+ }
+
// allow users to specify some environment variables
Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
@@ -223,8 +340,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
// Add Xmx for am memory
JAVA_OPTS += "-Xmx" + amMemory + "m "
- JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(),
- YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
+ JAVA_OPTS += " -Djava.io.tmpdir=" +
+ new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
// Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out.
@@ -240,6 +357,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
}
+
if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
}
@@ -254,7 +372,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
val commands = List[String](javaCommand +
" -server " +
JAVA_OPTS +
- " spark.deploy.yarn.ApplicationMaster" +
+ " org.apache.spark.deploy.yarn.ApplicationMaster" +
" --class " + args.userClass +
" --jar " + args.userJar +
userArgsToString(args) +
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 6cbfadc23b..852dbd7dab 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -24,6 +24,9 @@ import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
// TODO: Add code and support for ensuring that yarn resource 'asks' are location aware !
class ClientArguments(val args: Array[String]) {
+ var addJars: String = null
+ var files: String = null
+ var archives: String = null
var userJar: String = null
var userClass: String = null
var userArgs: Seq[String] = Seq[String]()
@@ -32,6 +35,7 @@ class ClientArguments(val args: Array[String]) {
var numWorkers = 2
var amQueue = System.getProperty("QUEUE", "default")
var amMemory: Int = 512
+ var appName: String = "Spark"
// TODO
var inputFormatInfo: List[InputFormatInfo] = null
@@ -78,6 +82,21 @@ class ClientArguments(val args: Array[String]) {
amQueue = value
args = tail
+ case ("--name") :: value :: tail =>
+ appName = value
+
+ case ("--addJars") :: value :: tail =>
+ addJars = value
+ args = tail
+
+ case ("--files") :: value :: tail =>
+ files = value
+ args = tail
+
+ case ("--archives") :: value :: tail =>
+ archives = value
+ args = tail
+
case Nil =>
if (userJar == null || userClass == null) {
printUsageAndExit(1)
@@ -92,13 +111,13 @@ class ClientArguments(val args: Array[String]) {
inputFormatInfo = inputFormatMap.values.toList
}
-
+
def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
if (unknownParam != null) {
System.err.println("Unknown/unsupported param " + unknownParam)
}
System.err.println(
- "Usage: spark.deploy.yarn.Client [options] \n" +
+ "Usage: org.apache.spark.deploy.yarn.Client [options] \n" +
"Options:\n" +
" --jar JAR_PATH Path to your application's JAR file (required)\n" +
" --class CLASS_NAME Name of your application's main class (required)\n" +
@@ -108,9 +127,13 @@ class ClientArguments(val args: Array[String]) {
" --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" +
" --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
" --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" +
- " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')"
+ " --name NAME The name of your application (Default: Spark)\n" +
+ " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" +
+ " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" +
+ " --files files Comma separated list of files to be distributed with the job.\n" +
+ " --archives archives Comma separated list of archives to be distributed with the job."
)
System.exit(exitCode)
}
-
+
}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index 72dcf7178e..8dac9e02ac 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -37,7 +37,8 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import scala.collection.JavaConversions._
import scala.collection.mutable.HashMap
-import org.apache.spark.{Logging, Utils}
+import org.apache.spark.Logging
+import org.apache.spark.util.Utils
class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String,
slaveId: String, hostname: String, workerMemory: Int, workerCores: Int)
@@ -76,8 +77,9 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
}
- JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(),
- YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
+ JAVA_OPTS += " -Djava.io.tmpdir=" +
+ new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
+
// Commenting it out for now - so that people can refer to the properties if required. Remove it once cpuset version is pushed out.
// The context is, default gc for server class machines end up using all cores to do gc - hence if there are multiple containers in same
@@ -135,11 +137,26 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
startReq.setContainerLaunchContext(ctx)
cm.startContainer(startReq)
}
+
+ private def setupDistributedCache(file: String,
+ rtype: LocalResourceType,
+ localResources: HashMap[String, LocalResource],
+ timestamp: String,
+ size: String) = {
+ val uri = new URI(file)
+ val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+ amJarRsrc.setType(rtype)
+ amJarRsrc.setVisibility(LocalResourceVisibility.APPLICATION)
+ amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+ amJarRsrc.setTimestamp(timestamp.toLong)
+ amJarRsrc.setSize(size.toLong)
+ localResources(uri.getFragment()) = amJarRsrc
+ }
def prepareLocalResources: HashMap[String, LocalResource] = {
logInfo("Preparing Local resources")
- val locaResources = HashMap[String, LocalResource]()
+ val localResources = HashMap[String, LocalResource]()
// Spark JAR
val sparkJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
@@ -149,7 +166,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
new URI(System.getenv("SPARK_YARN_JAR_PATH"))))
sparkJarResource.setTimestamp(System.getenv("SPARK_YARN_JAR_TIMESTAMP").toLong)
sparkJarResource.setSize(System.getenv("SPARK_YARN_JAR_SIZE").toLong)
- locaResources("spark.jar") = sparkJarResource
+ localResources("spark.jar") = sparkJarResource
// User JAR
val userJarResource = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
userJarResource.setType(LocalResourceType.FILE)
@@ -158,7 +175,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
new URI(System.getenv("SPARK_YARN_USERJAR_PATH"))))
userJarResource.setTimestamp(System.getenv("SPARK_YARN_USERJAR_TIMESTAMP").toLong)
userJarResource.setSize(System.getenv("SPARK_YARN_USERJAR_SIZE").toLong)
- locaResources("app.jar") = userJarResource
+ localResources("app.jar") = userJarResource
// Log4j conf - if available
if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) {
@@ -169,26 +186,39 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
new URI(System.getenv("SPARK_YARN_LOG4J_PATH"))))
log4jConfResource.setTimestamp(System.getenv("SPARK_YARN_LOG4J_TIMESTAMP").toLong)
log4jConfResource.setSize(System.getenv("SPARK_YARN_LOG4J_SIZE").toLong)
- locaResources("log4j.properties") = log4jConfResource
+ localResources("log4j.properties") = log4jConfResource
+ }
+
+ if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
+ val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+ val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+ val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
+ for( i <- 0 to distFiles.length - 1) {
+ setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
+ fileSizes(i))
+ }
}
+ if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
+ val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
+ val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
+ val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
+ for( i <- 0 to distArchives.length - 1) {
+ setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources,
+ timeStamps(i), fileSizes(i))
+ }
+ }
- logInfo("Prepared Local resources " + locaResources)
- return locaResources
+ logInfo("Prepared Local resources " + localResources)
+ return localResources
}
def prepareEnvironment: HashMap[String, String] = {
val env = new HashMap[String, String]()
- // If log4j present, ensure ours overrides all others
- if (System.getenv("SPARK_YARN_LOG4J_PATH") != null) {
- // Which is correct ?
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./log4j.properties")
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./")
- }
-
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, "./*")
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, "$CLASSPATH")
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name,
+ Environment.PWD.$() + Path.SEPARATOR + "*")
Client.populateHadoopClasspath(yarnConf, env)
// allow users to specify some environment variables
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 26ff214e12..6d6ef149cc 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -17,7 +17,8 @@
package org.apache.spark.deploy.yarn
-import org.apache.spark.{Logging, Utils}
+import org.apache.spark.Logging
+import org.apache.spark.util.Utils
import org.apache.spark.scheduler.SplitInfo
import scala.collection
import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId, ContainerId, Priority, Resource, ResourceRequest, ContainerStatus, Container}
@@ -479,6 +480,15 @@ object YarnAllocationHandler {
private val hostToRack = new ConcurrentHashMap[String, String]()
private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]()
+
+ def newAllocator(conf: Configuration,
+ resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
+ args: ApplicationMasterArguments): YarnAllocationHandler = {
+
+ new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers,
+ args.workerMemory, args.workerCores, Map[String, Int](), Map[String, Int]())
+ }
+
def newAllocator(conf: Configuration,
resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
args: ApplicationMasterArguments,
@@ -486,7 +496,6 @@ object YarnAllocationHandler {
val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
-
new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers,
args.workerMemory, args.workerCores, hostToCount, rackToCount)
}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
index 3828ddfc4f..29b3f22e13 100644
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
@@ -19,6 +19,7 @@ package org.apache.spark.scheduler.cluster
import org.apache.spark._
import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnAllocationHandler}
+import org.apache.spark.util.Utils
import org.apache.hadoop.conf.Configuration
/**
@@ -27,6 +28,8 @@ import org.apache.hadoop.conf.Configuration
*/
private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
+ logInfo("Created YarnClusterScheduler")
+
def this(sc: SparkContext) = this(sc, new Configuration())
// Nothing else for now ... initialize application master : which needs sparkContext to determine how to allocate