aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--.gitignore4
-rw-r--r--assembly/lib/PY4J_LICENSE.txt27
-rw-r--r--assembly/lib/PY4J_VERSION.txt1
-rw-r--r--assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jarbin103286 -> 0 bytes
-rw-r--r--assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom9
-rw-r--r--assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml12
-rw-r--r--assembly/pom.xml36
-rw-r--r--assembly/src/main/assembly/assembly.xml11
-rw-r--r--bagel/pom.xml12
-rw-r--r--bin/compute-classpath.cmd4
-rwxr-xr-xbin/compute-classpath.sh4
-rwxr-xr-xbin/pyspark (renamed from pyspark)14
-rw-r--r--bin/pyspark.cmd (renamed from pyspark.cmd)0
-rw-r--r--bin/pyspark2.cmd (renamed from pyspark2.cmd)4
-rwxr-xr-xbin/run-example (renamed from run-example)23
-rw-r--r--bin/run-example.cmd (renamed from run-example.cmd)0
-rw-r--r--bin/run-example2.cmd (renamed from run-example2.cmd)6
-rwxr-xr-xbin/spark-class (renamed from spark-class)29
-rw-r--r--bin/spark-class.cmd (renamed from spark-class.cmd)0
-rw-r--r--bin/spark-class2.cmd (renamed from spark-class2.cmd)8
-rwxr-xr-xbin/spark-shell (renamed from spark-shell)21
-rw-r--r--bin/spark-shell.cmd (renamed from spark-shell.cmd)5
-rwxr-xr-xconf/spark-env.sh.template2
-rw-r--r--core/pom.xml433
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileClient.java39
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java8
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java12
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileServer.java38
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java7
-rw-r--r--core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java22
-rwxr-xr-xcore/src/main/java/org/apache/spark/network/netty/PathResolver.java52
-rw-r--r--core/src/main/resources/org/apache/spark/log4j-defaults.properties8
-rw-r--r--core/src/main/scala/org/apache/spark/Accumulators.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/FutureAction.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/HttpServer.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/Logging.scala43
-rw-r--r--core/src/main/scala/org/apache/spark/MapOutputTracker.scala37
-rw-r--r--core/src/main/scala/org/apache/spark/Partitioner.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/SparkConf.scala198
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala598
-rw-r--r--core/src/main/scala/org/apache/spark/SparkEnv.scala71
-rw-r--r--core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/TaskEndReason.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/TaskState.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala49
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala113
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala54
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala116
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java1
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/Function.java8
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/Function2.java8
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/Function3.java8
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java12
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java12
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala165
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala49
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala45
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala20
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/Client.scala61
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/Master.scala122
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala13
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala60
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala27
-rw-r--r--core/src/main/scala/org/apache/spark/executor/Executor.scala61
-rw-r--r--core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala28
-rw-r--r--core/src/main/scala/org/apache/spark/io/CompressionCodec.scala19
-rw-r--r--core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/network/ConnectionManager.scala30
-rw-r--r--core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/network/ReceiverTest.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/network/SenderTest.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala41
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala127
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala12
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala82
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala110
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/RDD.scala130
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala19
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala9
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala427
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala)2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Pool.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala)7
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala29
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala20
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala)16
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala11
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala)76
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala691
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala (renamed from core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala)2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala712
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala51
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala16
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala25
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala19
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala109
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala219
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala191
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/Serializer.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala65
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala20
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala26
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala35
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageLevel.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala19
-rw-r--r--core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/ui/SparkUI.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala21
-rw-r--r--core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala15
-rw-r--r--core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala31
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala27
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala90
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala37
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala80
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/util/AkkaUtils.scala111
-rw-r--r--core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala68
-rw-r--r--core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala38
-rw-r--r--core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala50
-rw-r--r--core/src/main/scala/org/apache/spark/util/SizeEstimator.scala14
-rw-r--r--core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala30
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala117
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala4
-rw-r--r--core/src/test/resources/spark.conf8
-rw-r--r--core/src/test/resources/uncommons-maths-1.2.2.jarbin49019 -> 0 bytes
-rw-r--r--core/src/test/scala/org/apache/spark/AccumulatorSuite.scala32
-rw-r--r--core/src/test/scala/org/apache/spark/CheckpointSuite.scala366
-rw-r--r--core/src/test/scala/org/apache/spark/DistributedSuite.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/DriverSuite.scala10
-rw-r--r--core/src/test/scala/org/apache/spark/FailureSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/FileServerSuite.scala108
-rw-r--r--core/src/test/scala/org/apache/spark/JavaAPISuite.java83
-rw-r--r--core/src/test/scala/org/apache/spark/JobCancellationSuite.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala23
-rw-r--r--core/src/test/scala/org/apache/spark/SharedSparkContext.scala4
-rw-r--r--core/src/test/scala/org/apache/spark/SparkConfSuite.scala110
-rw-r--r--core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala132
-rw-r--r--core/src/test/scala/org/apache/spark/UnpersistSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala23
-rw-r--r--core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala8
-rw-r--r--core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala8
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala26
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala271
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala34
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala48
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala (renamed from core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala)18
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala66
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala (renamed from core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala)3
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala11
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala23
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala (renamed from core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala)34
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala (renamed from core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala)35
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala227
-rw-r--r--core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala33
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala101
-rw-r--r--core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala27
-rw-r--r--core/src/test/scala/org/apache/spark/ui/UISuite.scala1
-rw-r--r--core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala73
-rw-r--r--core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala74
-rw-r--r--core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala33
-rw-r--r--core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala37
-rw-r--r--core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala (renamed from core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala)31
-rw-r--r--data/kmeans_data.txt (renamed from kmeans_data.txt)0
-rw-r--r--data/lr_data.txt (renamed from lr_data.txt)0
-rw-r--r--data/pagerank_data.txt (renamed from pagerank_data.txt)0
-rw-r--r--docs/_config.yml4
-rwxr-xr-xdocs/_layouts/global.html8
-rw-r--r--docs/_plugins/copy_api_dirs.rb2
-rw-r--r--docs/bagel-programming-guide.md6
-rw-r--r--docs/building-with-maven.md8
-rw-r--r--docs/cluster-overview.md2
-rw-r--r--docs/configuration.md175
-rw-r--r--docs/css/bootstrap.min.css2
-rw-r--r--docs/hadoop-third-party-distributions.md3
-rw-r--r--docs/index.md18
-rw-r--r--docs/java-programming-guide.md4
-rw-r--r--docs/job-scheduling.md26
-rw-r--r--docs/mllib-guide.md330
-rw-r--r--docs/monitoring.md3
-rw-r--r--docs/python-programming-guide.md43
-rw-r--r--docs/quick-start.md66
-rw-r--r--docs/running-on-mesos.md19
-rw-r--r--docs/running-on-yarn.md54
-rw-r--r--docs/scala-programming-guide.md20
-rw-r--r--docs/spark-debugger.md2
-rw-r--r--docs/spark-standalone.md51
-rw-r--r--docs/streaming-programming-guide.md14
-rw-r--r--docs/tuning.md22
-rwxr-xr-xec2/spark_ec2.py6
-rw-r--r--examples/pom.xml67
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java31
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaKMeans.java25
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java24
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaPageRank.java16
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java12
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaTC.java18
-rw-r--r--examples/src/main/java/org/apache/spark/examples/JavaWordCount.java14
-rw-r--r--examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java21
-rw-r--r--examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java18
-rw-r--r--examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java21
-rw-r--r--examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java18
-rw-r--r--examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java28
-rw-r--r--examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java18
-rw-r--r--examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java13
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/LogQuery.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkALS.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkLR.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkPi.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkTC.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala10
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala8
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala16
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala5
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala7
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala18
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala8
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala6
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala5
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala23
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala6
-rw-r--r--external/flume/pom.xml93
-rw-r--r--external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala)8
-rw-r--r--external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala70
-rw-r--r--external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java34
-rw-r--r--external/flume/src/test/resources/log4j.properties29
-rw-r--r--external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala86
-rw-r--r--external/kafka/pom.xml97
-rw-r--r--external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala)34
-rw-r--r--external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala153
-rw-r--r--external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java45
-rw-r--r--external/kafka/src/test/resources/log4j.properties29
-rw-r--r--external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala39
-rw-r--r--external/mqtt/pom.xml108
-rw-r--r--external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala)29
-rw-r--r--external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala75
-rw-r--r--external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java37
-rw-r--r--external/mqtt/src/test/resources/log4j.properties29
-rw-r--r--external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala36
-rw-r--r--external/twitter/pom.xml89
-rw-r--r--external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala)11
-rw-r--r--external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala126
-rw-r--r--external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java46
-rw-r--r--external/twitter/src/test/resources/log4j.properties29
-rw-r--r--external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala43
-rw-r--r--external/zeromq/pom.xml89
-rw-r--r--external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala)16
-rw-r--r--external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala126
-rw-r--r--external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java50
-rw-r--r--external/zeromq/src/test/resources/log4j.properties29
-rw-r--r--external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala44
-rwxr-xr-xmake-distribution.sh27
-rw-r--r--mllib/pom.xml12
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala262
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala119
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala13
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala38
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala2
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java4
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java2
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala108
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala116
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala33
-rw-r--r--pom.xml185
-rw-r--r--project/SparkBuild.scala261
-rw-r--r--project/build.properties1
-rw-r--r--python/epydoc.conf4
-rw-r--r--python/lib/py4j-0.8.1-src.zipbin0 -> 37662 bytes
-rw-r--r--python/lib/py4j0.7.eggbin191756 -> 0 bytes
-rw-r--r--python/pyspark/__init__.py34
-rw-r--r--python/pyspark/accumulators.py6
-rw-r--r--python/pyspark/broadcast.py11
-rw-r--r--python/pyspark/conf.py171
-rw-r--r--python/pyspark/context.py136
-rw-r--r--python/pyspark/java_gateway.py4
-rw-r--r--python/pyspark/mllib/__init__.py20
-rw-r--r--python/pyspark/mllib/_common.py252
-rw-r--r--python/pyspark/mllib/classification.py86
-rw-r--r--python/pyspark/mllib/clustering.py79
-rw-r--r--python/pyspark/mllib/recommendation.py84
-rw-r--r--python/pyspark/mllib/regression.py110
-rw-r--r--python/pyspark/rdd.py170
-rw-r--r--python/pyspark/serializers.py299
-rw-r--r--python/pyspark/shell.py4
-rw-r--r--python/pyspark/tests.py22
-rw-r--r--python/pyspark/worker.py44
-rwxr-xr-xpython/run-tests4
-rwxr-xr-xpython/test_support/userlibrary.py17
-rw-r--r--repl-bin/pom.xml8
-rwxr-xr-xrepl-bin/src/deb/bin/run5
-rwxr-xr-xrepl-bin/src/deb/bin/spark-executor2
-rwxr-xr-xrepl-bin/src/deb/bin/spark-shell2
-rw-r--r--repl/lib/scala-jline.jarbin158463 -> 0 bytes
-rw-r--r--repl/pom.xml19
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/Main.scala8
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala (renamed from core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala)30
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala109
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala844
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala151
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala1686
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala63
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkImports.scala108
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala206
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala65
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala109
-rw-r--r--repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala32
-rw-r--r--repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala178
-rwxr-xr-xsbin/slaves.sh (renamed from bin/slaves.sh)6
-rwxr-xr-xsbin/spark-config.sh (renamed from bin/spark-config.sh)0
-rwxr-xr-xsbin/spark-daemon.sh (renamed from bin/spark-daemon.sh)8
-rwxr-xr-xsbin/spark-daemons.sh (renamed from bin/spark-daemons.sh)8
-rwxr-xr-xsbin/spark-executor (renamed from spark-executor)5
-rwxr-xr-xsbin/start-all.sh (renamed from bin/start-all.sh)10
-rwxr-xr-xsbin/start-master.sh (renamed from bin/start-master.sh)8
-rwxr-xr-xsbin/start-slave.sh (renamed from bin/start-slave.sh)6
-rwxr-xr-xsbin/start-slaves.sh (renamed from bin/start-slaves.sh)10
-rwxr-xr-xsbin/stop-all.sh (renamed from bin/stop-all.sh)10
-rwxr-xr-xsbin/stop-master.sh (renamed from bin/stop-master.sh)8
-rwxr-xr-xsbin/stop-slaves.sh (renamed from bin/stop-slaves.sh)10
-rwxr-xr-xsbt/sbt51
-rw-r--r--sbt/sbt-launch-0.11.3-2.jarbin1096763 -> 0 bytes
-rw-r--r--sbt/sbt.cmd25
-rw-r--r--streaming/pom.xml102
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala68
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStream.scala52
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala88
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala76
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala286
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala8
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala79
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala111
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala352
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala159
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala22
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala4
-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/dstream/ReducedWindowedDStream.scala9
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala12
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala3
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala5
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala23
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala35
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala55
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/Job.scala)24
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala)107
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala106
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala68
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala (renamed from streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala)9
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala75
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala81
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala48
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala4
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java174
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala22
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java46
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala20
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala109
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala13
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala100
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala71
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala73
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala19
-rw-r--r--tools/pom.xml12
-rw-r--r--tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala4
-rw-r--r--yarn/README.md12
-rw-r--r--yarn/alpha/pom.xml32
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala)255
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala)230
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala250
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala)90
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala)371
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala)0
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala)57
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala)0
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala (renamed from yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala)0
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala48
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala112
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala (renamed from yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala)3
-rw-r--r--yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala (renamed from yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala)0
-rw-r--r--yarn/pom.xml88
-rw-r--r--yarn/stable/pom.xml32
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala432
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala525
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala230
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala210
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala695
496 files changed, 17474 insertions, 8303 deletions
diff --git a/.gitignore b/.gitignore
index e1f64a1133..39635d7eef 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,7 +1,10 @@
*~
*.swp
+*.ipr
*.iml
+*.iws
.idea/
+sbt/*.jar
.settings
.cache
/build/
@@ -41,3 +44,4 @@ derby.log
dist/
spark-*-bin.tar.gz
unit-tests.log
+lib/
diff --git a/assembly/lib/PY4J_LICENSE.txt b/assembly/lib/PY4J_LICENSE.txt
deleted file mode 100644
index a70279ca14..0000000000
--- a/assembly/lib/PY4J_LICENSE.txt
+++ /dev/null
@@ -1,27 +0,0 @@
-
-Copyright (c) 2009-2011, Barthelemy Dagenais All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are met:
-
-- Redistributions of source code must retain the above copyright notice, this
-list of conditions and the following disclaimer.
-
-- Redistributions in binary form must reproduce the above copyright notice,
-this list of conditions and the following disclaimer in the documentation
-and/or other materials provided with the distribution.
-
-- The name of the author may not be used to endorse or promote products
-derived from this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
-AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
-IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
-ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE
-LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
-CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
-SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
-INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
-CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
-ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
-POSSIBILITY OF SUCH DAMAGE.
diff --git a/assembly/lib/PY4J_VERSION.txt b/assembly/lib/PY4J_VERSION.txt
deleted file mode 100644
index 04a0cd52a8..0000000000
--- a/assembly/lib/PY4J_VERSION.txt
+++ /dev/null
@@ -1 +0,0 @@
-b7924aabe9c5e63f0a4d8bbd17019534c7ec014e
diff --git a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar
deleted file mode 100644
index 73b7ddb7d1..0000000000
--- a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.jar
+++ /dev/null
Binary files differ
diff --git a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom b/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom
deleted file mode 100644
index 1c730e19b4..0000000000
--- a/assembly/lib/net/sf/py4j/py4j/0.7/py4j-0.7.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
- xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
- <modelVersion>4.0.0</modelVersion>
- <groupId>net.sf.py4j</groupId>
- <artifactId>py4j</artifactId>
- <version>0.7</version>
- <description>POM was created from install:install-file</description>
-</project>
diff --git a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml b/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml
deleted file mode 100644
index 6942ff45e7..0000000000
--- a/assembly/lib/net/sf/py4j/py4j/maven-metadata-local.xml
+++ /dev/null
@@ -1,12 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<metadata>
- <groupId>net.sf.py4j</groupId>
- <artifactId>py4j</artifactId>
- <versioning>
- <release>0.7</release>
- <versions>
- <version>0.7</version>
- </versions>
- <lastUpdated>20130828020333</lastUpdated>
- </versioning>
-</metadata>
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 09df8c1fd7..54a25910ce 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-assembly_2.9.3</artifactId>
+ <artifactId>spark-assembly_2.10</artifactId>
<name>Spark Project Assembly</name>
<url>http://spark.incubator.apache.org/</url>
@@ -41,33 +41,33 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl_2.9.3</artifactId>
+ <artifactId>spark-repl_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>net.sf.py4j</groupId>
<artifactId>py4j</artifactId>
- <version>0.7</version>
+ <version>0.8.1</version>
</dependency>
</dependencies>
@@ -79,7 +79,7 @@
<artifactId>maven-shade-plugin</artifactId>
<configuration>
<shadedArtifactAttached>false</shadedArtifactAttached>
- <outputFile>${project.build.directory}/scala-${scala.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
+ <outputFile>${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
<artifactSet>
<includes>
<include>*:*</include>
@@ -108,12 +108,12 @@
<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>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+ <resource>log4j.properties</resource>
+ </transformer>
</transformers>
</configuration>
</execution>
@@ -124,11 +124,21 @@
<profiles>
<profile>
- <id>hadoop2-yarn</id>
+ <id>yarn-alpha</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-yarn-alpha_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <id>yarn</id>
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn_2.9.3</artifactId>
+ <artifactId>spark-yarn_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
</dependencies>
diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml
index 47d3fa93d0..6af383db65 100644
--- a/assembly/src/main/assembly/assembly.xml
+++ b/assembly/src/main/assembly/assembly.xml
@@ -39,23 +39,20 @@
</fileSet>
<fileSet>
<directory>
- ${project.parent.basedir}/bin/
+ ${project.parent.basedir}/sbin/
</directory>
- <outputDirectory>/bin</outputDirectory>
+ <outputDirectory>/sbin</outputDirectory>
<includes>
<include>**/*</include>
</includes>
</fileSet>
<fileSet>
<directory>
- ${project.parent.basedir}
+ ${project.parent.basedir}/bin/
</directory>
<outputDirectory>/bin</outputDirectory>
<includes>
- <include>run-example*</include>
- <include>spark-class*</include>
- <include>spark-shell*</include>
- <include>spark-executor*</include>
+ <include>**/*</include>
</includes>
</fileSet>
</fileSets>
diff --git a/bagel/pom.xml b/bagel/pom.xml
index 0e552c880f..cb8e79f225 100644
--- a/bagel/pom.xml
+++ b/bagel/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_2.10</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_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -43,18 +43,18 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd
index cf38188c4b..4f60bff19c 100644
--- a/bin/compute-classpath.cmd
+++ b/bin/compute-classpath.cmd
@@ -20,7 +20,7 @@ rem
rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
rem script and the ExecutorRunner in standalone cluster mode.
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
rem Figure out where the Spark framework is installed
set FWDIR=%~dp0..\
@@ -29,7 +29,7 @@ rem Load environment variables from conf\spark-env.cmd, if it exists
if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd"
rem Build up classpath
-set CLASSPATH=%SPARK_CLASSPATH%;%FWDIR%conf
+set CLASSPATH=%FWDIR%conf
if exist "%FWDIR%RELEASE" (
for %%d in ("%FWDIR%jars\spark-assembly*.jar") do (
set ASSEMBLY_JAR=%%d
diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh
index af22d85afc..c10725e708 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -20,13 +20,13 @@
# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
# script and the ExecutorRunner in standalone cluster mode.
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Figure out where Spark is installed
FWDIR="$(cd `dirname $0`/..; pwd)"
# Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
. $FWDIR/conf/spark-env.sh
fi
diff --git a/pyspark b/bin/pyspark
index 4941a36d0d..d6810f4686 100755
--- a/pyspark
+++ b/bin/pyspark
@@ -18,12 +18,12 @@
#
# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
# Export this as SPARK_HOME
export SPARK_HOME="$FWDIR"
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Exit if the user hasn't compiled Spark
if [ ! -f "$FWDIR/RELEASE" ]; then
@@ -37,7 +37,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then
fi
# Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
. $FWDIR/conf/spark-env.sh
fi
@@ -59,8 +59,12 @@ if [ -n "$IPYTHON_OPTS" ]; then
fi
if [[ "$IPYTHON" = "1" ]] ; then
- IPYTHON_OPTS=${IPYTHON_OPTS:--i}
- exec ipython "$IPYTHON_OPTS" -c "%run $PYTHONSTARTUP"
+ # IPython <1.0.0 doesn't honor PYTHONSTARTUP, while 1.0.0+ does.
+ # Hence we clear PYTHONSTARTUP and use the -c "%run $IPYTHONSTARTUP" command which works on all versions
+ # We also force interactive mode with "-i"
+ IPYTHONSTARTUP=$PYTHONSTARTUP
+ PYTHONSTARTUP=
+ exec ipython "$IPYTHON_OPTS" -i -c "%run $IPYTHONSTARTUP"
else
exec "$PYSPARK_PYTHON" "$@"
fi
diff --git a/pyspark.cmd b/bin/pyspark.cmd
index 7c26fbbac2..7c26fbbac2 100644
--- a/pyspark.cmd
+++ b/bin/pyspark.cmd
diff --git a/pyspark2.cmd b/bin/pyspark2.cmd
index f58e349643..95791095ec 100644
--- a/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@ -17,10 +17,10 @@ rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
rem Export this as SPARK_HOME
set SPARK_HOME=%FWDIR%
diff --git a/run-example b/bin/run-example
index 08ec717ca5..2e9d51440b 100755
--- a/run-example
+++ b/bin/run-example
@@ -17,16 +17,21 @@
# limitations under the License.
#
-SCALA_VERSION=2.9.3
+cygwin=false
+case "`uname`" in
+ CYGWIN*) cygwin=true;;
+esac
+
+SCALA_VERSION=2.10
# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
# Export this as SPARK_HOME
export SPARK_HOME="$FWDIR"
# Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
. $FWDIR/conf/spark-env.sh
fi
@@ -40,25 +45,25 @@ fi
EXAMPLES_DIR="$FWDIR"/examples
SPARK_EXAMPLES_JAR=""
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-9Tg].jar`
fi
-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-9Tg].jar`
-fi
if [[ -z $SPARK_EXAMPLES_JAR ]]; then
echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2
echo "You need to build Spark with sbt/sbt assembly before running this program" >&2
exit 1
fi
+
# Since the examples JAR ideally shouldn't include spark-core (that dependency should be
# "provided"), also add our standard Spark classpath, built using compute-classpath.sh.
CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
CLASSPATH="$SPARK_EXAMPLES_JAR:$CLASSPATH"
+if $cygwin; then
+ CLASSPATH=`cygpath -wp $CLASSPATH`
+ export SPARK_EXAMPLES_JAR=`cygpath -w $SPARK_EXAMPLES_JAR`
+fi
+
# Find java binary
if [ -n "${JAVA_HOME}" ]; then
RUNNER="${JAVA_HOME}/bin/java"
diff --git a/run-example.cmd b/bin/run-example.cmd
index 5b2d048d6e..5b2d048d6e 100644
--- a/run-example.cmd
+++ b/bin/run-example.cmd
diff --git a/run-example2.cmd b/bin/run-example2.cmd
index dbb371ecfc..6861334cb0 100644
--- a/run-example2.cmd
+++ b/bin/run-example2.cmd
@@ -17,10 +17,10 @@ rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
rem Export this as SPARK_HOME
set SPARK_HOME=%FWDIR%
@@ -49,7 +49,7 @@ if "x%SPARK_EXAMPLES_JAR%"=="x" (
rem Compute Spark classpath using external script
set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
set DONT_PRINT_CLASSPATH=0
set CLASSPATH=%SPARK_EXAMPLES_JAR%;%CLASSPATH%
diff --git a/spark-class b/bin/spark-class
index bbeca7f245..c4225a392d 100755
--- a/spark-class
+++ b/bin/spark-class
@@ -17,16 +17,21 @@
# limitations under the License.
#
-SCALA_VERSION=2.9.3
+cygwin=false
+case "`uname`" in
+ CYGWIN*) cygwin=true;;
+esac
+
+SCALA_VERSION=2.10
# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
+FWDIR="$(cd `dirname $0`/..; pwd)"
# Export this as SPARK_HOME
export SPARK_HOME="$FWDIR"
# Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
+if [ -e "$FWDIR/conf/spark-env.sh" ] ; then
. $FWDIR/conf/spark-env.sh
fi
@@ -55,7 +60,7 @@ case "$1" in
'org.apache.spark.deploy.worker.Worker')
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
;;
- 'org.apache.spark.executor.StandaloneExecutorBackend')
+ 'org.apache.spark.executor.CoarseGrainedExecutorBackend')
OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
;;
'org.apache.spark.executor.MesosExecutorBackend')
@@ -87,7 +92,7 @@ JAVA_OPTS="$OUR_JAVA_OPTS"
JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM"
# Load extra JAVA_OPTS from conf/java-opts, if it exists
-if [ -e $FWDIR/conf/java-opts ] ; then
+if [ -e "$FWDIR/conf/java-opts" ] ; then
JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
fi
export JAVA_OPTS
@@ -124,7 +129,17 @@ fi
# Compute classpath using external script
CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
-CLASSPATH="$SPARK_TOOLS_JAR:$CLASSPATH"
+
+if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
+ CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR"
+fi
+
+if $cygwin; then
+ CLASSPATH=`cygpath -wp $CLASSPATH`
+ if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then
+ export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR`
+ fi
+fi
export CLASSPATH
if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
@@ -135,3 +150,5 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then
fi
exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@"
+
+
diff --git a/spark-class.cmd b/bin/spark-class.cmd
index 19850db9e1..19850db9e1 100644
--- a/spark-class.cmd
+++ b/bin/spark-class.cmd
diff --git a/spark-class2.cmd b/bin/spark-class2.cmd
index 3869d0761b..460e661476 100644
--- a/spark-class2.cmd
+++ b/bin/spark-class2.cmd
@@ -17,10 +17,10 @@ rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
-set SCALA_VERSION=2.9.3
+set SCALA_VERSION=2.10
rem Figure out where the Spark framework is installed
-set FWDIR=%~dp0
+set FWDIR=%~dp0..\
rem Export this as SPARK_HOME
set SPARK_HOME=%FWDIR%
@@ -73,9 +73,9 @@ for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar"
rem Compute classpath using external script
set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%bin\compute-classpath.cmd"
+call "%FWDIR%sbin\compute-classpath.cmd"
set DONT_PRINT_CLASSPATH=0
-set CLASSPATH=%SPARK_TOOLS_JAR%;%CLASSPATH%
+set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR%
rem Figure out where java is.
set RUNNER=java
diff --git a/spark-shell b/bin/spark-shell
index 9608bd3f30..e6885b51ef 100755
--- a/spark-shell
+++ b/bin/spark-shell
@@ -23,12 +23,16 @@
# if those two env vars are set in spark-env.sh but MASTER is not.
# Options:
# -c <cores> Set the number of cores for REPL to use
-#
+
+cygwin=false
+case "`uname`" in
+ CYGWIN*) cygwin=true;;
+esac
# Enter posix mode for bash
set -o posix
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
for o in "$@"; do
if [ "$1" = "-c" -o "$1" = "--cores" ]; then
@@ -79,7 +83,18 @@ if [[ ! $? ]]; then
saved_stty=""
fi
-$FWDIR/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
+if $cygwin; then
+ # Workaround for issue involving JLine and Cygwin
+ # (see http://sourceforge.net/p/jline/bugs/40/).
+ # If you're using the Mintty terminal emulator in Cygwin, may need to set the
+ # "Backspace sends ^H" setting in "Keys" section of the Mintty options
+ # (see https://github.com/sbt/sbt/issues/562).
+ stty -icanon min 1 -echo > /dev/null 2>&1
+ $FWDIR/bin/spark-class -Djline.terminal=unix $OPTIONS org.apache.spark.repl.Main "$@"
+ stty icanon echo > /dev/null 2>&1
+else
+ $FWDIR/bin/spark-class $OPTIONS org.apache.spark.repl.Main "$@"
+fi
# record the exit status lest it be overwritten:
# then reenable echo and propagate the code.
diff --git a/spark-shell.cmd b/bin/spark-shell.cmd
index 3e52bf835e..23973e3e3d 100644
--- a/spark-shell.cmd
+++ b/bin/spark-shell.cmd
@@ -17,6 +17,7 @@ rem See the License for the specific language governing permissions and
rem limitations under the License.
rem
-set FWDIR=%~dp0
+rem Find the path of sbin
+set SBIN=%~dp0..\sbin\
-cmd /V /E /C %FWDIR%spark-class2.cmd org.apache.spark.repl.Main %*
+cmd /V /E /C %SBIN%spark-class2.cmd org.apache.spark.repl.Main %*
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index 0a35ee7c79..e2071e2ade 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -18,4 +18,4 @@
# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g)
# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT
# - SPARK_WORKER_INSTANCES, to set the number of worker processes per node
-
+# - SPARK_WORKER_DIR, to set the working directory of worker processes
diff --git a/core/pom.xml b/core/pom.xml
index 6af229c71d..aac0a9d11e 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -17,226 +17,219 @@
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-parent</artifactId>
- <version>0.9.0-incubating-SNAPSHOT</version>
- <relativePath>../pom.xml</relativePath>
- </parent>
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-parent</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
- <packaging>jar</packaging>
- <name>Spark Project Core</name>
- <url>http://spark.incubator.apache.org/</url>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-core_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project Core</name>
+ <url>http://spark.incubator.apache.org/</url>
- <dependencies>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <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>
- <dependency>
- <groupId>org.apache.avro</groupId>
- <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>
- <dependency>
- <groupId>com.google.guava</groupId>
- <artifactId>guava</artifactId>
- </dependency>
- <dependency>
- <groupId>com.google.code.findbugs</groupId>
- <artifactId>jsr305</artifactId>
- </dependency>
- <dependency>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- </dependency>
- <dependency>
- <groupId>com.ning</groupId>
- <artifactId>compress-lzf</artifactId>
- </dependency>
- <dependency>
- <groupId>org.xerial.snappy</groupId>
- <artifactId>snappy-java</artifactId>
- </dependency>
- <dependency>
- <groupId>org.ow2.asm</groupId>
- <artifactId>asm</artifactId>
- </dependency>
- <dependency>
- <groupId>com.google.protobuf</groupId>
- <artifactId>protobuf-java</artifactId>
- </dependency>
- <dependency>
- <groupId>com.twitter</groupId>
- <artifactId>chill_2.9.3</artifactId>
- <version>0.3.1</version>
- </dependency>
- <dependency>
- <groupId>com.twitter</groupId>
- <artifactId>chill-java</artifactId>
- <version>0.3.1</version>
- </dependency>
- <dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-actor</artifactId>
- </dependency>
- <dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-remote</artifactId>
- </dependency>
- <dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-slf4j</artifactId>
- </dependency>
- <dependency>
- <groupId>org.scala-lang</groupId>
- <artifactId>scalap</artifactId>
- </dependency>
- <dependency>
- <groupId>org.scala-lang</groupId>
- <artifactId>scala-library</artifactId>
- </dependency>
- <dependency>
- <groupId>net.liftweb</groupId>
- <artifactId>lift-json_2.9.2</artifactId>
- </dependency>
- <dependency>
- <groupId>it.unimi.dsi</groupId>
- <artifactId>fastutil</artifactId>
- </dependency>
- <dependency>
- <groupId>colt</groupId>
- <artifactId>colt</artifactId>
- </dependency>
- <dependency>
- <groupId>com.github.scala-incubator.io</groupId>
- <artifactId>scala-io-file_2.9.2</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.mesos</groupId>
- <artifactId>mesos</artifactId>
- </dependency>
- <dependency>
- <groupId>io.netty</groupId>
- <artifactId>netty-all</artifactId>
- </dependency>
- <dependency>
- <groupId>log4j</groupId>
- <artifactId>log4j</artifactId>
- </dependency>
- <dependency>
- <groupId>com.codahale.metrics</groupId>
- <artifactId>metrics-core</artifactId>
- </dependency>
- <dependency>
- <groupId>com.codahale.metrics</groupId>
- <artifactId>metrics-jvm</artifactId>
- </dependency>
- <dependency>
- <groupId>com.codahale.metrics</groupId>
- <artifactId>metrics-json</artifactId>
- </dependency>
- <dependency>
- <groupId>com.codahale.metrics</groupId>
- <artifactId>metrics-ganglia</artifactId>
- </dependency>
- <dependency>
- <groupId>com.codahale.metrics</groupId>
- <artifactId>metrics-graphite</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.derby</groupId>
- <artifactId>derby</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.easymock</groupId>
- <artifactId>easymock</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>com.novocode</groupId>
- <artifactId>junit-interface</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-log4j12</artifactId>
- <scope>test</scope>
- </dependency>
- </dependencies>
- <build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-antrun-plugin</artifactId>
- <executions>
- <execution>
- <phase>test</phase>
- <goals>
- <goal>run</goal>
- </goals>
- <configuration>
- <exportAntProperties>true</exportAntProperties>
- <tasks>
- <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" />
- </or>
- </not>
- </condition>
- </fail>
- </tasks>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.scalatest</groupId>
- <artifactId>scalatest-maven-plugin</artifactId>
- <configuration>
- <environmentVariables>
- <SPARK_HOME>${basedir}/..</SPARK_HOME>
- <SPARK_TESTING>1</SPARK_TESTING>
- <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
- </environmentVariables>
- </configuration>
- </plugin>
- </plugins>
- </build>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <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>
+ <dependency>
+ <groupId>org.apache.avro</groupId>
+ <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>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.code.findbugs</groupId>
+ <artifactId>jsr305</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.ning</groupId>
+ <artifactId>compress-lzf</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.xerial.snappy</groupId>
+ <artifactId>snappy-java</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.ow2.asm</groupId>
+ <artifactId>asm</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.twitter</groupId>
+ <artifactId>chill_${scala.binary.version}</artifactId>
+ <version>0.3.1</version>
+ </dependency>
+ <dependency>
+ <groupId>com.twitter</groupId>
+ <artifactId>chill-java</artifactId>
+ <version>0.3.1</version>
+ </dependency>
+ <dependency>
+ <groupId>${akka.group}</groupId>
+ <artifactId>akka-remote_${scala.binary.version}</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>${akka.group}</groupId>
+ <artifactId>akka-slf4j_${scala.binary.version}</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.scala-lang</groupId>
+ <artifactId>scala-library</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>net.liftweb</groupId>
+ <artifactId>lift-json_${scala.binary.version}</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>it.unimi.dsi</groupId>
+ <artifactId>fastutil</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>colt</groupId>
+ <artifactId>colt</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.mesos</groupId>
+ <artifactId>mesos</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-all</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>log4j</groupId>
+ <artifactId>log4j</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.clearspring.analytics</groupId>
+ <artifactId>stream</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.codahale.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.codahale.metrics</groupId>
+ <artifactId>metrics-jvm</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.codahale.metrics</groupId>
+ <artifactId>metrics-json</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.codahale.metrics</groupId>
+ <artifactId>metrics-ganglia</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.codahale.metrics</groupId>
+ <artifactId>metrics-graphite</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.derby</groupId>
+ <artifactId>derby</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.scalacheck</groupId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.easymock</groupId>
+ <artifactId>easymock</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.novocode</groupId>
+ <artifactId>junit-interface</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-antrun-plugin</artifactId>
+ <executions>
+ <execution>
+ <phase>test</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <exportAntProperties>true</exportAntProperties>
+ <tasks>
+ <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" />
+ </or>
+ </not>
+ </condition>
+ </fail>
+ </tasks>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest-maven-plugin</artifactId>
+ <configuration>
+ <environmentVariables>
+ <SPARK_HOME>${basedir}/..</SPARK_HOME>
+ <SPARK_TESTING>1</SPARK_TESTING>
+ <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
+ </environmentVariables>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
</project>
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
index 20a7a3aa8c..d2d778b756 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClient.java
@@ -19,31 +19,36 @@ package org.apache.spark.network.netty;
import io.netty.bootstrap.Bootstrap;
import io.netty.channel.Channel;
-import io.netty.channel.ChannelFuture;
-import io.netty.channel.ChannelFutureListener;
import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
import io.netty.channel.oio.OioEventLoopGroup;
import io.netty.channel.socket.oio.OioSocketChannel;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.concurrent.TimeUnit;
+
class FileClient {
- private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());
- private FileClientHandler handler = null;
+ private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName());
+
+ private final FileClientHandler handler;
private Channel channel = null;
private Bootstrap bootstrap = null;
- private int connectTimeout = 60*1000; // 1 min
+ private EventLoopGroup group = null;
+ private final int connectTimeout;
+ private final int sendTimeout = 60; // 1 min
- public FileClient(FileClientHandler handler, int connectTimeout) {
+ FileClient(FileClientHandler handler, int connectTimeout) {
this.handler = handler;
this.connectTimeout = connectTimeout;
}
public void init() {
+ group = new OioEventLoopGroup();
bootstrap = new Bootstrap();
- bootstrap.group(new OioEventLoopGroup())
+ bootstrap.group(group)
.channel(OioSocketChannel.class)
.option(ChannelOption.SO_KEEPALIVE, true)
.option(ChannelOption.TCP_NODELAY, true)
@@ -58,6 +63,7 @@ class FileClient {
// ChannelFuture cf = channel.closeFuture();
//cf.addListener(new ChannelCloseListener(this));
} catch (InterruptedException e) {
+ LOG.warn("FileClient interrupted while trying to connect", e);
close();
}
}
@@ -73,16 +79,21 @@ class FileClient {
public void sendRequest(String file) {
//assert(file == null);
//assert(channel == null);
- channel.write(file + "\r\n");
+ try {
+ // Should be able to send the message to network link channel.
+ boolean bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS);
+ if (!bSent) {
+ throw new RuntimeException("Failed to send");
+ }
+ } catch (InterruptedException e) {
+ LOG.error("Error", e);
+ }
}
public void close() {
- if(channel != null) {
- channel.close();
- channel = null;
- }
- if ( bootstrap!=null) {
- bootstrap.shutdown();
+ if (group != null) {
+ group.shutdownGracefully();
+ group = null;
bootstrap = null;
}
}
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
index 65ee15d63b..264cf97d02 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java
@@ -17,17 +17,15 @@
package org.apache.spark.network.netty;
-import io.netty.buffer.BufType;
import io.netty.channel.ChannelInitializer;
import io.netty.channel.socket.SocketChannel;
import io.netty.handler.codec.string.StringEncoder;
-
class FileClientChannelInitializer extends ChannelInitializer<SocketChannel> {
- private FileClientHandler fhandler;
+ private final FileClientHandler fhandler;
- public FileClientChannelInitializer(FileClientHandler handler) {
+ FileClientChannelInitializer(FileClientHandler handler) {
fhandler = handler;
}
@@ -35,7 +33,7 @@ class FileClientChannelInitializer extends ChannelInitializer<SocketChannel> {
public void initChannel(SocketChannel channel) {
// file no more than 2G
channel.pipeline()
- .addLast("encoder", new StringEncoder(BufType.BYTE))
+ .addLast("encoder", new StringEncoder())
.addLast("handler", fhandler);
}
}
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 8a09210245..63d3d92725 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
@@ -19,11 +19,11 @@ package org.apache.spark.network.netty;
import io.netty.buffer.ByteBuf;
import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundByteHandlerAdapter;
+import io.netty.channel.SimpleChannelInboundHandler;
import org.apache.spark.storage.BlockId;
-abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
+abstract class FileClientHandler extends SimpleChannelInboundHandler<ByteBuf> {
private FileHeader currentHeader = null;
@@ -37,13 +37,7 @@ abstract class FileClientHandler extends ChannelInboundByteHandlerAdapter {
public abstract void handleError(BlockId blockId);
@Override
- public ByteBuf newInboundBuffer(ChannelHandlerContext ctx) {
- // Use direct buffer if possible.
- return ctx.alloc().ioBuffer();
- }
-
- @Override
- public void inboundBufferUpdated(ChannelHandlerContext ctx, ByteBuf in) {
+ public void channelRead0(ChannelHandlerContext ctx, ByteBuf in) {
// get header
if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) {
currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE()));
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
index 666432474d..c93425e278 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServer.java
@@ -20,34 +20,35 @@ package org.apache.spark.network.netty;
import java.net.InetSocketAddress;
import io.netty.bootstrap.ServerBootstrap;
-import io.netty.channel.Channel;
import io.netty.channel.ChannelFuture;
import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
import io.netty.channel.oio.OioEventLoopGroup;
import io.netty.channel.socket.oio.OioServerSocketChannel;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
/**
* Server that accept the path of a file an echo back its content.
*/
class FileServer {
- private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());
+ private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName());
- private ServerBootstrap bootstrap = null;
+ private EventLoopGroup bossGroup = null;
+ private EventLoopGroup workerGroup = null;
private ChannelFuture channelFuture = null;
private int port = 0;
- private Thread blockingThread = null;
- public FileServer(PathResolver pResolver, int port) {
+ FileServer(PathResolver pResolver, int port) {
InetSocketAddress addr = new InetSocketAddress(port);
// Configure the server.
- bootstrap = new ServerBootstrap();
- bootstrap.group(new OioEventLoopGroup(), new OioEventLoopGroup())
+ bossGroup = new OioEventLoopGroup();
+ workerGroup = new OioEventLoopGroup();
+
+ ServerBootstrap bootstrap = new ServerBootstrap();
+ bootstrap.group(bossGroup, workerGroup)
.channel(OioServerSocketChannel.class)
.option(ChannelOption.SO_BACKLOG, 100)
.option(ChannelOption.SO_RCVBUF, 1500)
@@ -68,7 +69,8 @@ class FileServer {
* Start the file server asynchronously in a new thread.
*/
public void start() {
- blockingThread = new Thread() {
+ Thread blockingThread = new Thread() {
+ @Override
public void run() {
try {
channelFuture.channel().closeFuture().sync();
@@ -90,13 +92,19 @@ class FileServer {
public void stop() {
// Close the bound channel.
if (channelFuture != null) {
- channelFuture.channel().close();
+ channelFuture.channel().close().awaitUninterruptibly();
channelFuture = null;
}
- // Shutdown bootstrap.
- if (bootstrap != null) {
- bootstrap.shutdown();
- bootstrap = null;
+
+ // Shutdown event groups
+ if (bossGroup != null) {
+ bossGroup.shutdownGracefully();
+ bossGroup = null;
+ }
+
+ if (workerGroup != null) {
+ workerGroup.shutdownGracefully();
+ workerGroup = null;
}
// TODO: Shutdown all accepted channels as well ?
}
diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
index 833af1632d..46efec8f8d 100644
--- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
+++ b/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java
@@ -23,12 +23,11 @@ import io.netty.handler.codec.DelimiterBasedFrameDecoder;
import io.netty.handler.codec.Delimiters;
import io.netty.handler.codec.string.StringDecoder;
-
class FileServerChannelInitializer extends ChannelInitializer<SocketChannel> {
- PathResolver pResolver;
+ private final PathResolver pResolver;
- public FileServerChannelInitializer(PathResolver pResolver) {
+ FileServerChannelInitializer(PathResolver pResolver) {
this.pResolver = pResolver;
}
@@ -36,7 +35,7 @@ class FileServerChannelInitializer extends ChannelInitializer<SocketChannel> {
public void initChannel(SocketChannel channel) {
channel.pipeline()
.addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter()))
- .addLast("strDecoder", new StringDecoder())
+ .addLast("stringDecoder", new StringDecoder())
.addLast("handler", new FileServerHandler(pResolver));
}
}
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 172c6e4b1c..3ac045f944 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
@@ -21,22 +21,26 @@ import java.io.File;
import java.io.FileInputStream;
import io.netty.channel.ChannelHandlerContext;
-import io.netty.channel.ChannelInboundMessageHandlerAdapter;
+import io.netty.channel.SimpleChannelInboundHandler;
import io.netty.channel.DefaultFileRegion;
import org.apache.spark.storage.BlockId;
import org.apache.spark.storage.FileSegment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
-class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
+class FileServerHandler extends SimpleChannelInboundHandler<String> {
- PathResolver pResolver;
+ private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName());
- public FileServerHandler(PathResolver pResolver){
+ private final PathResolver pResolver;
+
+ FileServerHandler(PathResolver pResolver){
this.pResolver = pResolver;
}
@Override
- public void messageReceived(ChannelHandlerContext ctx, String blockIdString) {
+ public void channelRead0(ChannelHandlerContext ctx, String blockIdString) {
BlockId blockId = BlockId.apply(blockIdString);
FileSegment fileSegment = pResolver.getBlockLocation(blockId);
// if getBlockLocation returns null, close the channel
@@ -57,13 +61,13 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
ctx.flush();
return;
}
- int len = new Long(length).intValue();
+ int len = (int) length;
ctx.write((new FileHeader(len, blockId)).buffer());
try {
- ctx.sendFile(new DefaultFileRegion(new FileInputStream(file)
+ ctx.write(new DefaultFileRegion(new FileInputStream(file)
.getChannel(), fileSegment.offset(), fileSegment.length()));
} catch (Exception e) {
- e.printStackTrace();
+ LOG.error("Exception: ", e);
}
} else {
ctx.write(new FileHeader(0, blockId).buffer());
@@ -73,7 +77,7 @@ class FileServerHandler extends ChannelInboundMessageHandlerAdapter<String> {
@Override
public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
- cause.printStackTrace();
+ LOG.error("Exception: ", cause);
ctx.close();
}
}
diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
index 9f7ced44cf..7ad8d03efb 100755
--- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
+++ b/core/src/main/java/org/apache/spark/network/netty/PathResolver.java
@@ -1,26 +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.network.netty;
-
-import org.apache.spark.storage.BlockId;
-import org.apache.spark.storage.FileSegment;
-
-public interface PathResolver {
- /** Get the file segment in which the given block resides. */
- public FileSegment getBlockLocation(BlockId blockId);
-}
+/*
+ * 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.network.netty;
+
+import org.apache.spark.storage.BlockId;
+import org.apache.spark.storage.FileSegment;
+
+public interface PathResolver {
+ /** Get the file segment in which the given block resides. */
+ FileSegment getBlockLocation(BlockId blockId);
+}
diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
new file mode 100644
index 0000000000..d72dbadc39
--- /dev/null
+++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
@@ -0,0 +1,8 @@
+# Set everything to be logged to the console
+log4j.rootCategory=INFO, console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala
index 6e922a612a..5f73d234aa 100644
--- a/core/src/main/scala/org/apache/spark/Accumulators.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulators.scala
@@ -41,7 +41,7 @@ class Accumulable[R, T] (
@transient initialValue: R,
param: AccumulableParam[R, T])
extends Serializable {
-
+
val id = Accumulators.newId
@transient private var value_ = initialValue // Current value on master
val zero = param.zero(initialValue) // Zero value to be passed to workers
@@ -113,7 +113,7 @@ class Accumulable[R, T] (
def setValue(newValue: R) {
this.value = newValue
}
-
+
// Called by Java when deserializing an object
private def readObject(in: ObjectInputStream) {
in.defaultReadObject()
@@ -177,7 +177,7 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
def zero(initialValue: R): R = {
// We need to clone initialValue, but it's hard to specify that R should also be Cloneable.
// Instead we'll serialize it to a buffer and load it back.
- val ser = new JavaSerializer().newInstance()
+ val ser = new JavaSerializer(new SparkConf(false)).newInstance()
val copy = ser.deserialize[R](ser.serialize(initialValue))
copy.clear() // In case it contained stuff
copy
@@ -215,7 +215,7 @@ private object Accumulators {
val originals = Map[Long, Accumulable[_, _]]()
val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]()
var lastId: Long = 0
-
+
def newId: Long = synchronized {
lastId += 1
return lastId
diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala
index 1ad9240cfa..c6b4ac5192 100644
--- a/core/src/main/scala/org/apache/spark/FutureAction.scala
+++ b/core/src/main/scala/org/apache/spark/FutureAction.scala
@@ -99,7 +99,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc:
override def ready(atMost: Duration)(implicit permit: CanAwait): SimpleFutureAction.this.type = {
if (!atMost.isFinite()) {
awaitResult()
- } else {
+ } else jobWaiter.synchronized {
val finishTime = System.currentTimeMillis() + atMost.toMillis
while (!isCompleted) {
val time = System.currentTimeMillis()
diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala
index cdfc9dd54e..69a738dc44 100644
--- a/core/src/main/scala/org/apache/spark/HttpServer.scala
+++ b/core/src/main/scala/org/apache/spark/HttpServer.scala
@@ -46,6 +46,7 @@ private[spark] class HttpServer(resourceBase: File) extends Logging {
if (server != null) {
throw new ServerStateException("Server is already started")
} else {
+ logInfo("Starting HTTP Server")
server = new Server()
val connector = new SocketConnector
connector.setMaxIdleTime(60*1000)
diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala
index 6a973ea495..4a34989e50 100644
--- a/core/src/main/scala/org/apache/spark/Logging.scala
+++ b/core/src/main/scala/org/apache/spark/Logging.scala
@@ -17,8 +17,8 @@
package org.apache.spark
-import org.slf4j.Logger
-import org.slf4j.LoggerFactory
+import org.apache.log4j.{LogManager, PropertyConfigurator}
+import org.slf4j.{Logger, LoggerFactory}
/**
* Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
@@ -33,6 +33,7 @@ trait Logging {
// Method to get or create the logger for this object
protected def log: Logger = {
if (log_ == null) {
+ initializeIfNecessary()
var className = this.getClass.getName
// Ignore trailing $'s in the class names for Scala objects
if (className.endsWith("$")) {
@@ -89,7 +90,39 @@ trait Logging {
log.isTraceEnabled
}
- // Method for ensuring that logging is initialized, to avoid having multiple
- // threads do it concurrently (as SLF4J initialization is not thread safe).
- protected def initLogging() { log }
+ private def initializeIfNecessary() {
+ if (!Logging.initialized) {
+ Logging.initLock.synchronized {
+ if (!Logging.initialized) {
+ initializeLogging()
+ }
+ }
+ }
+ }
+
+ private def initializeLogging() {
+ // If Log4j doesn't seem initialized, load a default properties file
+ val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
+ if (!log4jInitialized) {
+ val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
+ val classLoader = this.getClass.getClassLoader
+ Option(classLoader.getResource(defaultLogProps)) match {
+ case Some(url) =>
+ PropertyConfigurator.configure(url)
+ log.info(s"Using Spark's default log4j profile: $defaultLogProps")
+ case None =>
+ System.err.println(s"Spark was unable to load $defaultLogProps")
+ }
+ }
+ Logging.initialized = true
+
+ // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads
+ // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
+ log
+ }
+}
+
+object Logging {
+ @volatile private var initialized = false
+ val initLock = new Object()
}
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 5e465fa22c..77b8ca1cce 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -21,17 +21,15 @@ import java.io._
import java.util.zip.{GZIPInputStream, GZIPOutputStream}
import scala.collection.mutable.HashSet
+import scala.concurrent.Await
+import scala.concurrent.duration._
import akka.actor._
-import akka.dispatch._
import akka.pattern.ask
-import akka.util.Duration
-
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage.BlockManagerId
-import org.apache.spark.util.{MetadataCleanerType, Utils, MetadataCleaner, TimeStampedHashMap}
-
+import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils}
private[spark] sealed trait MapOutputTrackerMessage
private[spark] case class GetMapOutputStatuses(shuffleId: Int, requester: String)
@@ -52,10 +50,10 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster
}
}
-private[spark] class MapOutputTracker extends Logging {
+private[spark] class MapOutputTracker(conf: SparkConf) extends Logging {
+
+ private val timeout = AkkaUtils.askTimeout(conf)
- private val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
-
// Set to the MapOutputTrackerActor living on the driver
var trackerActor: ActorRef = _
@@ -67,14 +65,14 @@ private[spark] class MapOutputTracker extends Logging {
protected val epochLock = new java.lang.Object
private val metadataCleaner =
- new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup)
+ new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf)
// Send a message to the trackerActor and get its result within a default timeout, or
// throw a SparkException if this fails.
private def askTracker(message: Any): Any = {
try {
val future = trackerActor.ask(message)(timeout)
- return Await.result(future, timeout)
+ Await.result(future, timeout)
} catch {
case e: Exception =>
throw new SparkException("Error communicating with MapOutputTracker", e)
@@ -117,11 +115,11 @@ private[spark] class MapOutputTracker extends Logging {
fetching += shuffleId
}
}
-
+
if (fetchedStatuses == null) {
// We won the race to fetch the output locs; do so
logInfo("Doing the fetch; tracker actor = " + trackerActor)
- val hostPort = Utils.localHostPort()
+ val hostPort = Utils.localHostPort(conf)
// This try-finally prevents hangs due to timeouts:
try {
val fetchedBytes =
@@ -144,7 +142,7 @@ private[spark] class MapOutputTracker extends Logging {
else{
throw new FetchFailedException(null, shuffleId, -1, reduceId,
new Exception("Missing all output locations for shuffle " + shuffleId))
- }
+ }
} else {
statuses.synchronized {
return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, statuses)
@@ -184,7 +182,8 @@ private[spark] class MapOutputTracker extends Logging {
}
}
-private[spark] class MapOutputTrackerMaster extends MapOutputTracker {
+private[spark] class MapOutputTrackerMaster(conf: SparkConf)
+ extends MapOutputTracker(conf) {
// Cache a serialized version of the output statuses for each shuffle to send them out faster
private var cacheEpoch = epoch
@@ -244,12 +243,12 @@ private[spark] class MapOutputTrackerMaster extends MapOutputTracker {
case Some(bytes) =>
return bytes
case None =>
- statuses = mapStatuses(shuffleId)
+ statuses = mapStatuses.getOrElse(shuffleId, Array[MapStatus]())
epochGotten = epoch
}
}
// If we got here, we failed to find the serialized locations in the cache, so we pulled
- // out a snapshot of the locations as "locs"; let's serialize and return that
+ // out a snapshot of the locations as "statuses"; let's serialize and return that
val bytes = MapOutputTracker.serializeMapStatuses(statuses)
logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length))
// Add them into the table only if the epoch hasn't changed while we were working
@@ -274,6 +273,10 @@ private[spark] class MapOutputTrackerMaster extends MapOutputTracker {
override def updateEpoch(newEpoch: Long) {
// This might be called on the MapOutputTrackerMaster if we're running in local mode.
}
+
+ def has(shuffleId: Int): Boolean = {
+ cachedSerializedStatuses.get(shuffleId).isDefined || mapStatuses.contains(shuffleId)
+ }
}
private[spark] object MapOutputTracker {
@@ -308,7 +311,7 @@ private[spark] object MapOutputTracker {
statuses: Array[MapStatus]): Array[(BlockManagerId, Long)] = {
assert (statuses != null)
statuses.map {
- status =>
+ status =>
if (status == null) {
throw new FetchFailedException(null, shuffleId, -1, reduceId,
new Exception("Missing an output location for shuffle " + shuffleId))
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index 0e2c987a59..31b0773bfe 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -17,8 +17,10 @@
package org.apache.spark
-import org.apache.spark.util.Utils
+import scala.reflect.ClassTag
+
import org.apache.spark.rdd.RDD
+import org.apache.spark.util.Utils
/**
* An object that defines how the elements in a key-value pair RDD are partitioned by key.
@@ -50,7 +52,7 @@ object Partitioner {
for (r <- bySize if r.partitioner != None) {
return r.partitioner.get
}
- if (System.getProperty("spark.default.parallelism") != null) {
+ if (rdd.context.conf.contains("spark.default.parallelism")) {
return new HashPartitioner(rdd.context.defaultParallelism)
} else {
return new HashPartitioner(bySize.head.partitions.size)
@@ -72,7 +74,7 @@ class HashPartitioner(partitions: Int) extends Partitioner {
case null => 0
case _ => Utils.nonNegativeMod(key.hashCode, numPartitions)
}
-
+
override def equals(other: Any): Boolean = other match {
case h: HashPartitioner =>
h.numPartitions == numPartitions
@@ -85,10 +87,10 @@ class HashPartitioner(partitions: Int) extends Partitioner {
* A [[org.apache.spark.Partitioner]] that partitions sortable records by range into roughly equal ranges.
* Determines the ranges by sampling the RDD passed in.
*/
-class RangePartitioner[K <% Ordered[K]: ClassManifest, V](
+class RangePartitioner[K <% Ordered[K]: ClassTag, V](
partitions: Int,
@transient rdd: RDD[_ <: Product2[K,V]],
- private val ascending: Boolean = true)
+ private val ascending: Boolean = true)
extends Partitioner {
// An array of upper bounds for the first (partitions - 1) partitions
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
new file mode 100644
index 0000000000..2de32231e8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -0,0 +1,198 @@
+package org.apache.spark
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.HashMap
+
+import com.typesafe.config.ConfigFactory
+
+/**
+ * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs.
+ *
+ * Most of the time, you would create a SparkConf object with `new SparkConf()`, which will load
+ * values from both the `spark.*` Java system properties and any `spark.conf` on your application's
+ * classpath (if it has one). In this case, system properties take priority over `spark.conf`, and
+ * any parameters you set directly on the `SparkConf` object take priority over both of those.
+ *
+ * For unit tests, you can also call `new SparkConf(false)` to skip loading external settings and
+ * get the same configuration no matter what is on the classpath.
+ *
+ * All setter methods in this class support chaining. For example, you can write
+ * `new SparkConf().setMaster("local").setAppName("My app")`.
+ *
+ * Note that once a SparkConf object is passed to Spark, it is cloned and can no longer be modified
+ * by the user. Spark does not support modifying the configuration at runtime.
+ *
+ * @param loadDefaults whether to load values from the system properties and classpath
+ */
+class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with Logging {
+
+ /** Create a SparkConf that loads defaults from system properties and the classpath */
+ def this() = this(true)
+
+ private val settings = new HashMap[String, String]()
+
+ if (loadDefaults) {
+ ConfigFactory.invalidateCaches()
+ val typesafeConfig = ConfigFactory.systemProperties()
+ .withFallback(ConfigFactory.parseResources("spark.conf"))
+ for (e <- typesafeConfig.entrySet().asScala if e.getKey.startsWith("spark.")) {
+ settings(e.getKey) = e.getValue.unwrapped.toString
+ }
+ }
+
+ /** Set a configuration variable. */
+ def set(key: String, value: String): SparkConf = {
+ if (key == null) {
+ throw new NullPointerException("null key")
+ }
+ if (value == null) {
+ throw new NullPointerException("null value")
+ }
+ settings(key) = value
+ this
+ }
+
+ /**
+ * The master URL to connect to, such as "local" to run locally with one thread, "local[4]" to
+ * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster.
+ */
+ def setMaster(master: String): SparkConf = {
+ set("spark.master", master)
+ }
+
+ /** Set a name for your application. Shown in the Spark web UI. */
+ def setAppName(name: String): SparkConf = {
+ set("spark.app.name", name)
+ }
+
+ /** Set JAR files to distribute to the cluster. */
+ def setJars(jars: Seq[String]): SparkConf = {
+ for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor")
+ set("spark.jars", jars.filter(_ != null).mkString(","))
+ }
+
+ /** Set JAR files to distribute to the cluster. (Java-friendly version.) */
+ def setJars(jars: Array[String]): SparkConf = {
+ setJars(jars.toSeq)
+ }
+
+ /**
+ * Set an environment variable to be used when launching executors for this application.
+ * These variables are stored as properties of the form spark.executorEnv.VAR_NAME
+ * (for example spark.executorEnv.PATH) but this method makes them easier to set.
+ */
+ def setExecutorEnv(variable: String, value: String): SparkConf = {
+ set("spark.executorEnv." + variable, value)
+ }
+
+ /**
+ * Set multiple environment variables to be used when launching executors.
+ * These variables are stored as properties of the form spark.executorEnv.VAR_NAME
+ * (for example spark.executorEnv.PATH) but this method makes them easier to set.
+ */
+ def setExecutorEnv(variables: Seq[(String, String)]): SparkConf = {
+ for ((k, v) <- variables) {
+ setExecutorEnv(k, v)
+ }
+ this
+ }
+
+ /**
+ * Set multiple environment variables to be used when launching executors.
+ * (Java-friendly version.)
+ */
+ def setExecutorEnv(variables: Array[(String, String)]): SparkConf = {
+ setExecutorEnv(variables.toSeq)
+ }
+
+ /**
+ * Set the location where Spark is installed on worker nodes.
+ */
+ def setSparkHome(home: String): SparkConf = {
+ set("spark.home", home)
+ }
+
+ /** Set multiple parameters together */
+ def setAll(settings: Traversable[(String, String)]) = {
+ this.settings ++= settings
+ this
+ }
+
+ /** Set a parameter if it isn't already configured */
+ def setIfMissing(key: String, value: String): SparkConf = {
+ if (!settings.contains(key)) {
+ settings(key) = value
+ }
+ this
+ }
+
+ /** Remove a parameter from the configuration */
+ def remove(key: String): SparkConf = {
+ settings.remove(key)
+ this
+ }
+
+ /** Get a parameter; throws a NoSuchElementException if it's not set */
+ def get(key: String): String = {
+ settings.getOrElse(key, throw new NoSuchElementException(key))
+ }
+
+ /** Get a parameter, falling back to a default if not set */
+ def get(key: String, defaultValue: String): String = {
+ settings.getOrElse(key, defaultValue)
+ }
+
+ /** Get a parameter as an Option */
+ def getOption(key: String): Option[String] = {
+ settings.get(key)
+ }
+
+ /** Get all parameters as a list of pairs */
+ def getAll: Array[(String, String)] = settings.clone().toArray
+
+ /** Get a parameter as an integer, falling back to a default if not set */
+ def getInt(key: String, defaultValue: Int): Int = {
+ getOption(key).map(_.toInt).getOrElse(defaultValue)
+ }
+
+ /** Get a parameter as a long, falling back to a default if not set */
+ def getLong(key: String, defaultValue: Long): Long = {
+ getOption(key).map(_.toLong).getOrElse(defaultValue)
+ }
+
+ /** Get a parameter as a double, falling back to a default if not set */
+ def getDouble(key: String, defaultValue: Double): Double = {
+ getOption(key).map(_.toDouble).getOrElse(defaultValue)
+ }
+
+ /** Get a parameter as a boolean, falling back to a default if not set */
+ def getBoolean(key: String, defaultValue: Boolean): Boolean = {
+ getOption(key).map(_.toBoolean).getOrElse(defaultValue)
+ }
+
+ /** Get all executor environment variables set on this SparkConf */
+ def getExecutorEnv: Seq[(String, String)] = {
+ val prefix = "spark.executorEnv."
+ getAll.filter{case (k, v) => k.startsWith(prefix)}
+ .map{case (k, v) => (k.substring(prefix.length), v)}
+ }
+
+ /** Get all akka conf variables set on this SparkConf */
+ def getAkkaConf: Seq[(String, String)] = getAll.filter {case (k, v) => k.startsWith("akka.")}
+
+ /** Does the configuration contain a given parameter? */
+ def contains(key: String): Boolean = settings.contains(key)
+
+ /** Copy this object */
+ override def clone: SparkConf = {
+ new SparkConf(false).setAll(settings)
+ }
+
+ /**
+ * Return a string listing all keys and values, one per line. This is useful to print the
+ * configuration out for debugging.
+ */
+ def toDebugString: String = {
+ settings.toArray.sorted.map{case (k, v) => k + "=" + v}.mkString("\n")
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 42b2985b50..0e47f4e442 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -19,35 +19,23 @@ package org.apache.spark
import java.io._
import java.net.URI
-import java.util.Properties
+import java.util.{UUID, Properties}
import java.util.concurrent.atomic.AtomicInteger
-import scala.collection.Map
+import scala.collection.{Map, Set}
import scala.collection.generic.Growable
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
+
+import scala.collection.mutable.{ArrayBuffer, HashMap}
+import scala.reflect.{ClassTag, classTag}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.ArrayWritable
-import org.apache.hadoop.io.BooleanWritable
-import org.apache.hadoop.io.BytesWritable
-import org.apache.hadoop.io.DoubleWritable
-import org.apache.hadoop.io.FloatWritable
-import org.apache.hadoop.io.IntWritable
-import org.apache.hadoop.io.LongWritable
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.io.Writable
-import org.apache.hadoop.mapred.FileInputFormat
-import org.apache.hadoop.mapred.InputFormat
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapred.SequenceFileInputFormat
-import org.apache.hadoop.mapred.TextInputFormat
-import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
+import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable,
+FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable}
+import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, SequenceFileInputFormat,
+TextInputFormat}
+import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob}
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
-
import org.apache.mesos.MesosNativeLibrary
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
@@ -55,59 +43,106 @@ import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd._
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend,
- SparkDeploySchedulerBackend, ClusterScheduler, SimrSchedulerBackend}
+ SparkDeploySchedulerBackend, SimrSchedulerBackend}
import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
-import org.apache.spark.scheduler.local.LocalScheduler
-import org.apache.spark.scheduler.StageInfo
+import org.apache.spark.scheduler.local.LocalBackend
import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils}
import org.apache.spark.ui.SparkUI
-import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType,
- TimeStampedHashMap, Utils}
+import org.apache.spark.util.{Utils, TimeStampedHashMap, MetadataCleaner, MetadataCleanerType,
+ClosureCleaner}
/**
* Main entry point for Spark functionality. A SparkContext represents the connection to a Spark
* cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster.
*
- * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param appName A name for your application, to display on the cluster web UI.
- * @param sparkHome Location where Spark is installed on cluster nodes.
- * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
- * system or HDFS, HTTP, HTTPS, or FTP URLs.
- * @param environment Environment variables to set on worker nodes.
+ * @param config a Spark Config object describing the application configuration. Any settings in
+ * this config overrides the default configs as well as system properties.
+ * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can
+ * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]]
+ * from a list of input files or InputFormats for the application.
*/
class SparkContext(
- val master: String,
- val appName: String,
- 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())
+ config: SparkConf,
+ // 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. It contains
+ // a map from hostname to a list of input format splits on the host.
+ val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map())
extends Logging {
- // Ensure logging is initialized before we spawn any threads
- initLogging()
+ /**
+ * Alternative constructor that allows setting common Spark properties directly
+ *
+ * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+ * @param appName A name for your application, to display on the cluster web UI
+ * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
+ */
+ def this(master: String, appName: String, conf: SparkConf) =
+ this(SparkContext.updatedConf(conf, master, appName))
- // Set Spark driver host and port system properties
- if (System.getProperty("spark.driver.host") == null) {
- System.setProperty("spark.driver.host", Utils.localHostName())
+ /**
+ * Alternative constructor that allows setting common Spark properties directly
+ *
+ * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+ * @param appName A name for your application, to display on the cluster web UI.
+ * @param sparkHome Location where Spark is installed on cluster nodes.
+ * @param jars Collection of JARs to send to the cluster. These can be paths on the local file
+ * system or HDFS, HTTP, HTTPS, or FTP URLs.
+ * @param environment Environment variables to set on worker nodes.
+ */
+ def this(
+ master: String,
+ appName: String,
+ sparkHome: String = null,
+ jars: Seq[String] = Nil,
+ environment: Map[String, String] = Map(),
+ preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) =
+ {
+ this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment),
+ preferredNodeLocationData)
}
- if (System.getProperty("spark.driver.port") == null) {
- System.setProperty("spark.driver.port", "0")
+
+ private[spark] val conf = config.clone()
+
+ /**
+ * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be
+ * changed at runtime.
+ */
+ def getConf: SparkConf = conf.clone()
+
+ if (!conf.contains("spark.master")) {
+ throw new SparkException("A master URL must be set in your configuration")
+ }
+ if (!conf.contains("spark.app.name")) {
+ throw new SparkException("An application must be set in your configuration")
+ }
+
+ if (conf.get("spark.logConf", "false").toBoolean) {
+ logInfo("Spark configuration:\n" + conf.toDebugString)
}
+ // Set Spark driver host and port system properties
+ conf.setIfMissing("spark.driver.host", Utils.localHostName())
+ conf.setIfMissing("spark.driver.port", "0")
+
+ val jars: Seq[String] = if (conf.contains("spark.jars")) {
+ conf.get("spark.jars").split(",").filter(_.size != 0)
+ } else {
+ null
+ }
+
+ val master = conf.get("spark.master")
+ val appName = conf.get("spark.app.name")
+
val isLocal = (master == "local" || master.startsWith("local["))
// Create the Spark execution environment (cache, map output tracker, etc)
- private[spark] val env = SparkEnv.createFromSystemProperties(
+ private[spark] val env = SparkEnv.create(
+ conf,
"<driver>",
- System.getProperty("spark.driver.host"),
- System.getProperty("spark.driver.port").toInt,
- true,
- isLocal)
+ conf.get("spark.driver.host"),
+ conf.get("spark.driver.port").toInt,
+ isDriver = true,
+ isLocal = isLocal)
SparkEnv.set(env)
// Used to store a URL for each static file/jar together with the file's local timestamp
@@ -116,7 +151,8 @@ class SparkContext(
// Keeps track of all persisted RDDs
private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]]
- private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup)
+ private[spark] val metadataCleaner =
+ new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf)
// Initialize the Spark UI
private[spark] val ui = new SparkUI(this)
@@ -126,23 +162,30 @@ class SparkContext(
// Add each JAR given through the constructor
if (jars != null) {
- jars.foreach { addJar(_) }
+ jars.foreach(addJar)
}
+ private[spark] val executorMemory = conf.getOption("spark.executor.memory")
+ .orElse(Option(System.getenv("SPARK_MEM")))
+ .map(Utils.memoryStringToMb)
+ .getOrElse(512)
+
// Environment variables to pass to our executors
private[spark] val executorEnvs = HashMap[String, String]()
// Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner
- for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", "SPARK_TESTING")) {
- val value = System.getenv(key)
- if (value != null) {
- executorEnvs(key) = value
- }
+ for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS");
+ value <- Option(System.getenv(key))) {
+ executorEnvs(key) = value
}
- // Since memory can be set with a system property too, use that
- executorEnvs("SPARK_MEM") = SparkContext.executorMemoryRequested + "m"
- if (environment != null) {
- executorEnvs ++= environment
+ // Convert java options to env vars as a work around
+ // since we can't set env vars directly in sbt.
+ for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing"))
+ value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} {
+ executorEnvs(envKey) = value
}
+ // Since memory can be set with a system property too, use that
+ executorEnvs("SPARK_MEM") = executorMemory + "m"
+ executorEnvs ++= conf.getExecutorEnv
// Set SPARK_USER for user who is running SparkContext.
val sparkUser = Option {
@@ -153,122 +196,35 @@ class SparkContext(
executorEnvs("SPARK_USER") = sparkUser
// Create and start the scheduler
- 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
- val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r
- // 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
- // Regular expression for connection to Mesos cluster
- val MESOS_REGEX = """mesos://(.*)""".r
- // Regular expression for connection to Simr cluster
- val SIMR_REGEX = """simr://(.*)""".r
-
- master match {
- case "local" =>
- new LocalScheduler(1, 0, this)
-
- case LOCAL_N_REGEX(threads) =>
- new LocalScheduler(threads.toInt, 0, this)
-
- case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
- new LocalScheduler(threads.toInt, maxFailures.toInt, this)
-
- case SPARK_REGEX(sparkUrl) =>
- val scheduler = new ClusterScheduler(this)
- val masterUrls = sparkUrl.split(",").map("spark://" + _)
- val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName)
- scheduler.initialize(backend)
- scheduler
-
- case SIMR_REGEX(simrUrl) =>
- val scheduler = new ClusterScheduler(this)
- val backend = new SimrSchedulerBackend(scheduler, this, simrUrl)
- scheduler.initialize(backend)
- scheduler
-
- case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
- // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
- val memoryPerSlaveInt = memoryPerSlave.toInt
- if (SparkContext.executorMemoryRequested > memoryPerSlaveInt) {
- throw new SparkException(
- "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
- memoryPerSlaveInt, SparkContext.executorMemoryRequested))
- }
-
- val scheduler = new ClusterScheduler(this)
- val localCluster = new LocalSparkCluster(
- numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
- val masterUrls = localCluster.start()
- val backend = new SparkDeploySchedulerBackend(scheduler, this, masterUrls, appName)
- scheduler.initialize(backend)
- backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
- localCluster.stop()
- }
- scheduler
-
- case "yarn-standalone" =>
- val scheduler = try {
- val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
- val cons = clazz.getConstructor(classOf[SparkContext])
- cons.newInstance(this).asInstanceOf[ClusterScheduler]
- } catch {
- // TODO: Enumerate the exact reasons why it can fail
- // But irrespective of it, it means we cannot proceed !
- case th: Throwable => {
- throw new SparkException("YARN mode not available ?", th)
- }
- }
- val backend = new CoarseGrainedSchedulerBackend(scheduler, this.env.actorSystem)
- scheduler.initialize(backend)
- scheduler
-
- case MESOS_REGEX(mesosUrl) =>
- MesosNativeLibrary.load()
- val scheduler = new ClusterScheduler(this)
- val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean
- val backend = if (coarseGrained) {
- new CoarseMesosSchedulerBackend(scheduler, this, mesosUrl, appName)
- } else {
- new MesosSchedulerBackend(scheduler, this, mesosUrl, appName)
- }
- scheduler.initialize(backend)
- scheduler
-
- case _ =>
- throw new SparkException("Could not parse Master URL: '" + master + "'")
- }
- }
+ private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName)
taskScheduler.start()
@volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler)
+ dagScheduler.start()
ui.start()
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
val hadoopConfiguration = {
val env = SparkEnv.get
- val conf = SparkHadoopUtil.get.newConfiguration()
+ val hadoopConf = SparkHadoopUtil.get.newConfiguration()
// Explicitly check for S3 environment variables
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"))
- conf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
+ hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
+ hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID"))
+ hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
+ hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY"))
}
// Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar"
- Utils.getSystemProperties.foreach { case (key, value) =>
+ conf.getAll.foreach { case (key, value) =>
if (key.startsWith("spark.hadoop.")) {
- conf.set(key.substring("spark.hadoop.".length), value)
+ hadoopConf.set(key.substring("spark.hadoop.".length), value)
}
}
- val bufferSize = System.getProperty("spark.buffer.size", "65536")
- conf.set("io.file.buffer.size", bufferSize)
- conf
+ val bufferSize = conf.get("spark.buffer.size", "65536")
+ hadoopConf.set("io.file.buffer.size", bufferSize)
+ hadoopConf
}
private[spark] var checkpointDir: Option[String] = None
@@ -278,7 +234,7 @@ class SparkContext(
override protected def childValue(parent: Properties): Properties = new Properties(parent)
}
- private[spark] def getLocalProperties(): Properties = localProperties.get()
+ private[spark] def getLocalProperties: Properties = localProperties.get()
private[spark] def setLocalProperties(props: Properties) {
localProperties.set(props)
@@ -354,19 +310,19 @@ class SparkContext(
// Methods for creating RDDs
/** Distribute a local Scala collection to form an RDD. */
- def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
+ def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]())
}
/** Distribute a local Scala collection to form an RDD. */
- def makeRDD[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
+ def makeRDD[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
parallelize(seq, numSlices)
}
/** Distribute a local Scala collection to form an RDD, with one or more
* location preferences (hostnames of Spark nodes) for each object.
* Create a new partition for each collection item. */
- def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = {
+ def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = {
val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs)
}
@@ -419,7 +375,7 @@ class SparkContext(
}
/**
- * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys,
+ * Smarter version of hadoopFile() that uses class tags to figure out the classes of keys,
* values and the InputFormat so that users don't need to pass them directly. Instead, callers
* can just write, for example,
* {{{
@@ -427,17 +383,17 @@ class SparkContext(
* }}}
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int)
- (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F])
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F])
: RDD[(K, V)] = {
hadoopFile(path,
- fm.erasure.asInstanceOf[Class[F]],
- km.erasure.asInstanceOf[Class[K]],
- vm.erasure.asInstanceOf[Class[V]],
+ fm.runtimeClass.asInstanceOf[Class[F]],
+ km.runtimeClass.asInstanceOf[Class[K]],
+ vm.runtimeClass.asInstanceOf[Class[V]],
minSplits)
}
/**
- * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys,
+ * Smarter version of hadoopFile() that uses class tags to figure out the classes of keys,
* values and the InputFormat so that users don't need to pass them directly. Instead, callers
* can just write, for example,
* {{{
@@ -445,17 +401,17 @@ class SparkContext(
* }}}
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String)
- (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]): RDD[(K, V)] =
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] =
hadoopFile[K, V, F](path, defaultMinSplits)
/** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */
def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](path: String)
- (implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]): RDD[(K, V)] = {
+ (implicit km: ClassTag[K], vm: ClassTag[V], fm: ClassTag[F]): RDD[(K, V)] = {
newAPIHadoopFile(
path,
- fm.erasure.asInstanceOf[Class[F]],
- km.erasure.asInstanceOf[Class[K]],
- vm.erasure.asInstanceOf[Class[V]])
+ fm.runtimeClass.asInstanceOf[Class[F]],
+ km.runtimeClass.asInstanceOf[Class[K]],
+ vm.runtimeClass.asInstanceOf[Class[V]])
}
/**
@@ -513,11 +469,11 @@ class SparkContext(
* IntWritable). The most natural thing would've been to have implicit objects for the
* converters, but then we couldn't have an object for every subclass of Writable (you can't
* have a parameterized singleton object). We use functions instead to create a new converter
- * for the appropriate type. In addition, we pass the converter a ClassManifest of its type to
+ * for the appropriate type. In addition, we pass the converter a ClassTag of its type to
* allow it to figure out the Writable class to use in the subclass case.
*/
def sequenceFile[K, V](path: String, minSplits: Int = defaultMinSplits)
- (implicit km: ClassManifest[K], vm: ClassManifest[V],
+ (implicit km: ClassTag[K], vm: ClassTag[V],
kcf: () => WritableConverter[K], vcf: () => WritableConverter[V])
: RDD[(K, V)] = {
val kc = kcf()
@@ -536,7 +492,7 @@ class SparkContext(
* slow if you use the default serializer (Java serialization), though the nice thing about it is
* that there's very little effort required to save arbitrary objects.
*/
- def objectFile[T: ClassManifest](
+ def objectFile[T: ClassTag](
path: String,
minSplits: Int = defaultMinSplits
): RDD[T] = {
@@ -545,17 +501,17 @@ class SparkContext(
}
- protected[spark] def checkpointFile[T: ClassManifest](
+ protected[spark] def checkpointFile[T: ClassTag](
path: String
): RDD[T] = {
new CheckpointRDD[T](this, path)
}
/** Build the union of a list of RDDs. */
- def union[T: ClassManifest](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
+ def union[T: ClassTag](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
/** Build the union of a list of RDDs passed as variable-length arguments. */
- def union[T: ClassManifest](first: RDD[T], rest: RDD[T]*): RDD[T] =
+ def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] =
new UnionRDD(this, Seq(first) ++ rest)
// Methods for creating shared variables
@@ -608,10 +564,8 @@ class SparkContext(
}
addedFiles(key) = System.currentTimeMillis
- // Fetch the file locally in case a job is executed locally.
- // Jobs that run through LocalScheduler will already fetch the required dependencies,
- // but jobs run in DAGScheduler.runLocally() will not so we must fetch the files here.
- Utils.fetchFile(path, new File(SparkFiles.getRootDirectory))
+ // Fetch the file locally in case a job is executed using DAGScheduler.runLocally().
+ Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf)
logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
}
@@ -781,15 +735,27 @@ class SparkContext(
* (in that order of preference). If neither of these is set, return None.
*/
private[spark] def getSparkHome(): Option[String] = {
- if (sparkHome != null) {
- Some(sparkHome)
- } else if (System.getProperty("spark.home") != null) {
- Some(System.getProperty("spark.home"))
- } else if (System.getenv("SPARK_HOME") != null) {
- Some(System.getenv("SPARK_HOME"))
- } else {
- None
- }
+ conf.getOption("spark.home").orElse(Option(System.getenv("SPARK_HOME")))
+ }
+
+ /**
+ * Support function for API backtraces.
+ */
+ def setCallSite(site: String) {
+ setLocalProperty("externalCallSite", site)
+ }
+
+ /**
+ * Support function for API backtraces.
+ */
+ def clearCallSite() {
+ setLocalProperty("externalCallSite", null)
+ }
+
+ private[spark] def getCallSite(): String = {
+ val callSite = getLocalProperty("externalCallSite")
+ if (callSite == null) return Utils.formatSparkCallSite
+ callSite
}
/**
@@ -798,13 +764,13 @@ class SparkContext(
* flag specifies whether the scheduler can run the computation on the driver rather than
* shipping it out to the cluster, for short actions like first().
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
allowLocal: Boolean,
resultHandler: (Int, U) => Unit) {
- val callSite = Utils.formatSparkCallSite
+ val callSite = getCallSite
val cleanedFunc = clean(func)
logInfo("Starting job: " + callSite)
val start = System.nanoTime
@@ -819,7 +785,7 @@ class SparkContext(
* allowLocal flag specifies whether the scheduler can run the computation on the driver rather
* than shipping it out to the cluster, for short actions like first().
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
@@ -834,7 +800,7 @@ class SparkContext(
* Run a job on a given set of partitions of an RDD, but take a function of type
* `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`.
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: Iterator[T] => U,
partitions: Seq[Int],
@@ -846,21 +812,21 @@ class SparkContext(
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
- def runJob[T, U: ClassManifest](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = {
+ def runJob[T, U: ClassTag](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = {
runJob(rdd, func, 0 until rdd.partitions.size, false)
}
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
- def runJob[T, U: ClassManifest](rdd: RDD[T], func: Iterator[T] => U): Array[U] = {
+ def runJob[T, U: ClassTag](rdd: RDD[T], func: Iterator[T] => U): Array[U] = {
runJob(rdd, func, 0 until rdd.partitions.size, false)
}
/**
* Run a job on all partitions in an RDD and pass the results to a handler function.
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
processPartition: (TaskContext, Iterator[T]) => U,
resultHandler: (Int, U) => Unit)
@@ -871,7 +837,7 @@ class SparkContext(
/**
* Run a job on all partitions in an RDD and pass the results to a handler function.
*/
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
processPartition: Iterator[T] => U,
resultHandler: (Int, U) => Unit)
@@ -888,7 +854,7 @@ class SparkContext(
func: (TaskContext, Iterator[T]) => U,
evaluator: ApproximateEvaluator[U, R],
timeout: Long): PartialResult[R] = {
- val callSite = Utils.formatSparkCallSite
+ val callSite = getCallSite
logInfo("Starting job: " + callSite)
val start = System.nanoTime
val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout,
@@ -908,7 +874,7 @@ class SparkContext(
resultFunc: => R): SimpleFutureAction[R] =
{
val cleanF = clean(processPartition)
- val callSite = Utils.formatSparkCallSite
+ val callSite = getCallSite
val waiter = dagScheduler.submitJob(
rdd,
(context: TaskContext, iter: Iterator[T]) => cleanF(iter),
@@ -944,22 +910,15 @@ class SparkContext(
/**
* Set the directory under which RDDs are going to be checkpointed. The directory must
- * be a HDFS path if running on a cluster. If the directory does not exist, it will
- * be created. If the directory exists and useExisting is set to true, then the
- * exisiting directory will be used. Otherwise an exception will be thrown to
- * prevent accidental overriding of checkpoint files in the existing directory.
+ * be a HDFS path if running on a cluster.
*/
- def setCheckpointDir(dir: String, useExisting: Boolean = false) {
- val path = new Path(dir)
- val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration())
- if (!useExisting) {
- if (fs.exists(path)) {
- throw new Exception("Checkpoint directory '" + path + "' already exists.")
- } else {
- fs.mkdirs(path)
- }
+ def setCheckpointDir(directory: String) {
+ checkpointDir = Option(directory).map { dir =>
+ val path = new Path(dir, UUID.randomUUID().toString)
+ val fs = path.getFileSystem(hadoopConfiguration)
+ fs.mkdirs(path)
+ fs.getFileStatus(path).getPath().toString
}
- checkpointDir = Some(dir)
}
/** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */
@@ -1017,16 +976,16 @@ object SparkContext {
// TODO: Add AccumulatorParams for other types, e.g. lists and strings
- implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) =
+ implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) =
new PairRDDFunctions(rdd)
- implicit def rddToAsyncRDDActions[T: ClassManifest](rdd: RDD[T]) = new AsyncRDDActions(rdd)
+ implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd)
- implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest](
+ implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag](
rdd: RDD[(K, V)]) =
new SequenceFileRDDFunctions(rdd)
- implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
+ implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag](
rdd: RDD[(K, V)]) =
new OrderedRDDFunctions[K, V, (K, V)](rdd)
@@ -1051,16 +1010,16 @@ object SparkContext {
implicit def stringToText(s: String) = new Text(s)
- private implicit def arrayToArrayWritable[T <% Writable: ClassManifest](arr: Traversable[T]): ArrayWritable = {
+ private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]): ArrayWritable = {
def anyToWritable[U <% Writable](u: U): Writable = u
- new ArrayWritable(classManifest[T].erasure.asInstanceOf[Class[Writable]],
+ new ArrayWritable(classTag[T].runtimeClass.asInstanceOf[Class[Writable]],
arr.map(x => anyToWritable(x)).toArray)
}
// Helper objects for converting common types to Writable
- private def simpleWritableConverter[T, W <: Writable: ClassManifest](convert: W => T) = {
- val wClass = classManifest[W].erasure.asInstanceOf[Class[W]]
+ private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = {
+ val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]]
new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W]))
}
@@ -1079,11 +1038,11 @@ object SparkContext {
implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString)
implicit def writableWritableConverter[T <: Writable]() =
- new WritableConverter[T](_.erasure.asInstanceOf[Class[T]], _.asInstanceOf[T])
+ new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T])
/**
* Find the JAR from which a given class was loaded, to make it easy for users to pass
- * their JARs to SparkContext
+ * their JARs to SparkContext.
*/
def jarOfClass(cls: Class[_]): Seq[String] = {
val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class")
@@ -1100,28 +1059,181 @@ object SparkContext {
}
}
- /** Find the JAR that contains the class of a particular object */
+ /**
+ * Find the JAR that contains the class of a particular object, to make it easy for users
+ * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in
+ * your driver program.
+ */
def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass)
- /** Get the amount of memory per executor requested through system properties or SPARK_MEM */
- private[spark] val executorMemoryRequested = {
- // TODO: Might need to add some extra memory for the non-heap parts of the JVM
- Option(System.getProperty("spark.executor.memory"))
- .orElse(Option(System.getenv("SPARK_MEM")))
- .map(Utils.memoryStringToMb)
- .getOrElse(512)
+ /**
+ * Creates a modified version of a SparkConf with the parameters that can be passed separately
+ * to SparkContext, to make it easier to write SparkContext's constructors. This ignores
+ * parameters that are passed as the default value of null, instead of throwing an exception
+ * like SparkConf would.
+ */
+ private def updatedConf(
+ conf: SparkConf,
+ master: String,
+ appName: String,
+ sparkHome: String = null,
+ jars: Seq[String] = Nil,
+ environment: Map[String, String] = Map()): SparkConf =
+ {
+ val res = conf.clone()
+ res.setMaster(master)
+ res.setAppName(appName)
+ if (sparkHome != null) {
+ res.setSparkHome(sparkHome)
+ }
+ if (!jars.isEmpty) {
+ res.setJars(jars)
+ }
+ res.setExecutorEnv(environment.toSeq)
+ res
+ }
+
+ /** Creates a task scheduler based on a given master URL. Extracted for testing. */
+ private def createTaskScheduler(sc: SparkContext, master: String, appName: String)
+ : 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
+ val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r
+ // 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
+ // Regular expression for connection to Mesos cluster by mesos:// or zk:// url
+ val MESOS_REGEX = """(mesos|zk)://.*""".r
+ // Regular expression for connection to Simr cluster
+ val SIMR_REGEX = """simr://(.*)""".r
+
+ // When running locally, don't try to re-execute tasks on failure.
+ val MAX_LOCAL_TASK_FAILURES = 1
+
+ master match {
+ case "local" =>
+ val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true)
+ val backend = new LocalBackend(scheduler, 1)
+ scheduler.initialize(backend)
+ scheduler
+
+ case LOCAL_N_REGEX(threads) =>
+ val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true)
+ val backend = new LocalBackend(scheduler, threads.toInt)
+ scheduler.initialize(backend)
+ scheduler
+
+ case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
+ val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true)
+ val backend = new LocalBackend(scheduler, threads.toInt)
+ scheduler.initialize(backend)
+ scheduler
+
+ case SPARK_REGEX(sparkUrl) =>
+ val scheduler = new TaskSchedulerImpl(sc)
+ val masterUrls = sparkUrl.split(",").map("spark://" + _)
+ val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName)
+ scheduler.initialize(backend)
+ scheduler
+
+ case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
+ // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang.
+ val memoryPerSlaveInt = memoryPerSlave.toInt
+ if (sc.executorMemory > memoryPerSlaveInt) {
+ throw new SparkException(
+ "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format(
+ memoryPerSlaveInt, sc.executorMemory))
+ }
+
+ val scheduler = new TaskSchedulerImpl(sc)
+ val localCluster = new LocalSparkCluster(
+ numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
+ val masterUrls = localCluster.start()
+ val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName)
+ scheduler.initialize(backend)
+ backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
+ localCluster.stop()
+ }
+ scheduler
+
+ case "yarn-standalone" =>
+ val scheduler = try {
+ val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler")
+ val cons = clazz.getConstructor(classOf[SparkContext])
+ cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
+ } catch {
+ // TODO: Enumerate the exact reasons why it can fail
+ // But irrespective of it, it means we cannot proceed !
+ case th: Throwable => {
+ throw new SparkException("YARN mode not available ?", th)
+ }
+ }
+ val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
+ scheduler.initialize(backend)
+ scheduler
+
+ case "yarn-client" =>
+ val scheduler = try {
+ val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientClusterScheduler")
+ val cons = clazz.getConstructor(classOf[SparkContext])
+ cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl]
+
+ } catch {
+ case th: Throwable => {
+ throw new SparkException("YARN mode not available ?", th)
+ }
+ }
+
+ val backend = try {
+ val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend")
+ val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext])
+ cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend]
+ } catch {
+ case th: Throwable => {
+ throw new SparkException("YARN mode not available ?", th)
+ }
+ }
+
+ scheduler.initialize(backend)
+ scheduler
+
+ case mesosUrl @ MESOS_REGEX(_) =>
+ MesosNativeLibrary.load()
+ val scheduler = new TaskSchedulerImpl(sc)
+ val coarseGrained = sc.conf.get("spark.mesos.coarse", "false").toBoolean
+ val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs
+ val backend = if (coarseGrained) {
+ new CoarseMesosSchedulerBackend(scheduler, sc, url, appName)
+ } else {
+ new MesosSchedulerBackend(scheduler, sc, url, appName)
+ }
+ scheduler.initialize(backend)
+ scheduler
+
+ case SIMR_REGEX(simrUrl) =>
+ val scheduler = new TaskSchedulerImpl(sc)
+ val backend = new SimrSchedulerBackend(scheduler, sc, simrUrl)
+ scheduler.initialize(backend)
+ scheduler
+
+ case _ =>
+ throw new SparkException("Could not parse Master URL: '" + master + "'")
+ }
}
}
/**
* A class encapsulating how to convert some type T to Writable. It stores both the Writable class
* corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion.
- * The getter for the writable class takes a ClassManifest[T] in case this is a generic object
+ * The getter for the writable class takes a ClassTag[T] in case this is a generic object
* that doesn't know the type of T when it is created. This sounds strange but is necessary to
* support converting subclasses of Writable to themselves (writableWritableConverter).
*/
private[spark] class WritableConverter[T](
- val writableClass: ClassManifest[T] => Class[_ <: Writable],
+ val writableClass: ClassTag[T] => Class[_ <: Writable],
val convert: Writable => T)
extends Serializable
diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala
index ff2df8fb6a..2e36ccb9a0 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -17,11 +17,10 @@
package org.apache.spark
-import collection.mutable
-import serializer.Serializer
+import scala.collection.mutable
+import scala.concurrent.Await
-import akka.actor.{Actor, ActorRef, Props, ActorSystemImpl, ActorSystem}
-import akka.remote.RemoteActorRefProvider
+import akka.actor._
import org.apache.spark.broadcast.BroadcastManager
import org.apache.spark.metrics.MetricsSystem
@@ -40,7 +39,7 @@ import com.google.common.collect.MapMaker
* objects needs to have the right SparkEnv set. You can get the current environment with
* SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set.
*/
-class SparkEnv (
+class SparkEnv private[spark] (
val executorId: String,
val actorSystem: ActorSystem,
val serializerManager: SerializerManager,
@@ -54,7 +53,8 @@ class SparkEnv (
val connectionManager: ConnectionManager,
val httpFileServer: HttpFileServer,
val sparkFilesDir: String,
- val metricsSystem: MetricsSystem) {
+ val metricsSystem: MetricsSystem,
+ val conf: SparkConf) {
private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]()
@@ -62,7 +62,7 @@ class SparkEnv (
// (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]()
- def stop() {
+ private[spark] def stop() {
pythonWorkers.foreach { case(key, worker) => worker.stop() }
httpFileServer.stop()
mapOutputTracker.stop()
@@ -74,9 +74,11 @@ class SparkEnv (
actorSystem.shutdown()
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut
// down, but let's call it anyway in case it gets fixed in a later release
- actorSystem.awaitTermination()
+ // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it.
+ //actorSystem.awaitTermination()
}
+ private[spark]
def createPythonWorker(pythonExec: String, envVars: Map[String, String]): java.net.Socket = {
synchronized {
val key = (pythonExec, envVars)
@@ -105,33 +107,35 @@ object SparkEnv extends Logging {
/**
* Returns the ThreadLocal SparkEnv.
*/
- def getThreadLocal : SparkEnv = {
+ def getThreadLocal: SparkEnv = {
env.get()
}
- def createFromSystemProperties(
+ private[spark] def create(
+ conf: SparkConf,
executorId: String,
hostname: String,
port: Int,
isDriver: Boolean,
isLocal: Boolean): SparkEnv = {
- val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port)
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port,
+ conf = conf)
// Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port),
// figure out which port number Akka actually bound to and set spark.driver.port to it.
if (isDriver && port == 0) {
- System.setProperty("spark.driver.port", boundPort.toString)
+ conf.set("spark.driver.port", boundPort.toString)
}
// set only if unset until now.
- if (System.getProperty("spark.hostPort", null) == null) {
+ if (!conf.contains("spark.hostPort")) {
if (!isDriver){
// unexpected
Utils.logErrorWithStack("Unexpected NOT to have spark.hostPort set")
}
Utils.checkHost(hostname)
- System.setProperty("spark.hostPort", hostname + ":" + boundPort)
+ conf.set("spark.hostPort", hostname + ":" + boundPort)
}
val classLoader = Thread.currentThread.getContextClassLoader
@@ -139,49 +143,51 @@ object SparkEnv extends Logging {
// Create an instance of the class named by the given Java system property, or by
// defaultClassName if the property is not set, and return it as a T
def instantiateClass[T](propertyName: String, defaultClassName: String): T = {
- val name = System.getProperty(propertyName, defaultClassName)
+ val name = conf.get(propertyName, defaultClassName)
Class.forName(name, true, classLoader).newInstance().asInstanceOf[T]
}
val serializerManager = new SerializerManager
val serializer = serializerManager.setDefault(
- System.getProperty("spark.serializer", "org.apache.spark.serializer.JavaSerializer"))
+ conf.get("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf)
val closureSerializer = serializerManager.get(
- System.getProperty("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"))
+ conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"),
+ conf)
def registerOrLookup(name: String, newActor: => Actor): ActorRef = {
if (isDriver) {
logInfo("Registering " + name)
actorSystem.actorOf(Props(newActor), name = name)
} else {
- val driverHost: String = System.getProperty("spark.driver.host", "localhost")
- val driverPort: Int = System.getProperty("spark.driver.port", "7077").toInt
+ val driverHost: String = conf.get("spark.driver.host", "localhost")
+ val driverPort: Int = conf.get("spark.driver.port", "7077").toInt
Utils.checkHost(driverHost, "Expected hostname")
- val url = "akka://spark@%s:%s/user/%s".format(driverHost, driverPort, name)
- logInfo("Connecting to " + name + ": " + url)
- actorSystem.actorFor(url)
+ val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name"
+ val timeout = AkkaUtils.lookupTimeout(conf)
+ logInfo(s"Connecting to $name: $url")
+ Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout)
}
}
val blockManagerMaster = new BlockManagerMaster(registerOrLookup(
"BlockManagerMaster",
- new BlockManagerMasterActor(isLocal)))
- val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer)
+ new BlockManagerMasterActor(isLocal, conf)), conf)
+ val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf)
val connectionManager = blockManager.connectionManager
- val broadcastManager = new BroadcastManager(isDriver)
+ val broadcastManager = new BroadcastManager(isDriver, conf)
val cacheManager = new CacheManager(blockManager)
// Have to assign trackerActor after initialization as MapOutputTrackerActor
// requires the MapOutputTracker itself
val mapOutputTracker = if (isDriver) {
- new MapOutputTrackerMaster()
+ new MapOutputTrackerMaster(conf)
} else {
- new MapOutputTracker()
+ new MapOutputTracker(conf)
}
mapOutputTracker.trackerActor = registerOrLookup(
"MapOutputTracker",
@@ -192,12 +198,12 @@ object SparkEnv extends Logging {
val httpFileServer = new HttpFileServer()
httpFileServer.initialize()
- System.setProperty("spark.fileserver.uri", httpFileServer.serverUri)
+ conf.set("spark.fileserver.uri", httpFileServer.serverUri)
val metricsSystem = if (isDriver) {
- MetricsSystem.createMetricsSystem("driver")
+ MetricsSystem.createMetricsSystem("driver", conf)
} else {
- MetricsSystem.createMetricsSystem("executor")
+ MetricsSystem.createMetricsSystem("executor", conf)
}
metricsSystem.start()
@@ -211,7 +217,7 @@ object SparkEnv extends Logging {
}
// Warn about deprecated spark.cache.class property
- if (System.getProperty("spark.cache.class") != null) {
+ if (conf.contains("spark.cache.class")) {
logWarning("The spark.cache.class property is no longer being used! Specify storage " +
"levels using the RDD.persist() method instead.")
}
@@ -230,6 +236,7 @@ object SparkEnv extends Logging {
connectionManager,
httpFileServer,
sparkFilesDir,
- metricsSystem)
+ metricsSystem,
+ conf)
}
}
diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index 103a1c2051..618d95015f 100644
--- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
+++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -127,10 +127,6 @@ class SparkHadoopWriter(@transient jobConf: JobConf)
cmtr.commitJob(getJobContext())
}
- def cleanup() {
- getOutputCommitter().cleanupJob(getJobContext())
- }
-
// ********* Private Functions *********
private def getOutputFormat(): OutputFormat[AnyRef,AnyRef] = {
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index c1e5e04b31..faf6dcd618 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -53,5 +53,3 @@ private[spark] case class ExceptionFailure(
private[spark] case object TaskResultLost 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/TaskState.scala b/core/src/main/scala/org/apache/spark/TaskState.scala
index 19ce8369d9..0bf1e4a5e2 100644
--- a/core/src/main/scala/org/apache/spark/TaskState.scala
+++ b/core/src/main/scala/org/apache/spark/TaskState.scala
@@ -19,8 +19,7 @@ package org.apache.spark
import org.apache.mesos.Protos.{TaskState => MesosTaskState}
-private[spark] object TaskState
- extends Enumeration("LAUNCHING", "RUNNING", "FINISHED", "FAILED", "KILLED", "LOST") {
+private[spark] object TaskState extends Enumeration {
val LAUNCHING, RUNNING, FINISHED, FAILED, KILLED, LOST = Value
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
index 043cb183ba..da30cf619a 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala
@@ -17,18 +17,23 @@
package org.apache.spark.api.java
+import scala.reflect.ClassTag
+
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions
import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.util.StatCounter
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.storage.StorageLevel
+
import java.lang.Double
import org.apache.spark.Partitioner
+import scala.collection.JavaConverters._
+
class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, JavaDoubleRDD] {
- override val classManifest: ClassManifest[Double] = implicitly[ClassManifest[Double]]
+ override val classTag: ClassTag[Double] = implicitly[ClassTag[Double]]
override val rdd: RDD[Double] = srdd.map(x => Double.valueOf(x))
@@ -42,7 +47,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
/** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
def cache(): JavaDoubleRDD = fromRDD(srdd.cache())
- /**
+ /**
* Set this RDD's storage level to persist its values across operations after the first time
* it is computed. Can only be called once on each RDD.
*/
@@ -106,7 +111,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
/**
* Return an RDD with the elements from `this` that are not in `other`.
- *
+ *
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
@@ -182,6 +187,44 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
/** (Experimental) Approximate operation to return the sum within a timeout. */
def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout)
+
+ /**
+ * Compute a histogram of the data using bucketCount number of buckets evenly
+ * spaced between the minimum and maximum of the RDD. For example if the min
+ * value is 0 and the max is 100 and there are two buckets the resulting
+ * buckets will be [0,50) [50,100]. bucketCount must be at least 1
+ * If the RDD contains infinity, NaN throws an exception
+ * If the elements in RDD do not vary (max == min) always returns a single bucket.
+ */
+ def histogram(bucketCount: Int): Pair[Array[scala.Double], Array[Long]] = {
+ val result = srdd.histogram(bucketCount)
+ (result._1, result._2)
+ }
+
+ /**
+ * Compute a histogram using the provided buckets. The buckets are all open
+ * to the left except for the last which is closed
+ * e.g. for the array
+ * [1,10,20,50] the buckets are [1,10) [10,20) [20,50]
+ * e.g 1<=x<10 , 10<=x<20, 20<=x<50
+ * And on the input of 1 and 50 we would have a histogram of 1,0,0
+ *
+ * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched
+ * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets
+ * to true.
+ * buckets must be sorted and not contain any duplicates.
+ * buckets array must be at least two elements
+ * All NaN entries are treated the same. If you have a NaN bucket it must be
+ * the maximum value of the last position and all NaN entries will be counted
+ * in that bucket.
+ */
+ def histogram(buckets: Array[scala.Double]): Array[Long] = {
+ srdd.histogram(buckets, false)
+ }
+
+ def histogram(buckets: Array[Double], evenBuckets: Boolean): Array[Long] = {
+ srdd.histogram(buckets.map(_.toDouble), evenBuckets)
+ }
}
object JavaDoubleRDD {
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
index 2142fd7327..55c87450ac 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala
@@ -22,6 +22,7 @@ import java.util.Comparator
import scala.Tuple2
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import com.google.common.base.Optional
import org.apache.hadoop.io.compress.CompressionCodec
@@ -43,13 +44,13 @@ import org.apache.spark.rdd.OrderedRDDFunctions
import org.apache.spark.storage.StorageLevel
-class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManifest[K],
- implicit val vManifest: ClassManifest[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
+class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kClassTag: ClassTag[K],
+ implicit val vClassTag: ClassTag[V]) extends JavaRDDLike[(K, V), JavaPairRDD[K, V]] {
override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
- override val classManifest: ClassManifest[(K, V)] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+ override val classTag: ClassTag[(K, V)] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]]
import JavaPairRDD._
@@ -58,7 +59,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
def cache(): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.cache())
- /**
+ /**
* Set this RDD's storage level to persist its values across operations after the first time
* it is computed. Can only be called once on each RDD.
*/
@@ -138,14 +139,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
override def first(): (K, V) = rdd.first()
// Pair RDD functions
-
+
/**
- * Generic function to combine the elements for each key using a custom set of aggregation
- * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a
- * "combined type" C * Note that V and C can be different -- for example, one might group an
- * RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three
+ * Generic function to combine the elements for each key using a custom set of aggregation
+ * functions. Turns a JavaPairRDD[(K, V)] into a result of type JavaPairRDD[(K, C)], for a
+ * "combined type" C * Note that V and C can be different -- for example, one might group an
+ * RDD of type (Int, Int) into an RDD of type (Int, List[Int]). Users provide three
* functions:
- *
+ *
* - `createCombiner`, which turns a V into a C (e.g., creates a one-element list)
* - `mergeValue`, to merge a V into a C (e.g., adds it to the end of a list)
* - `mergeCombiners`, to combine two C's into a single one.
@@ -157,8 +158,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C],
partitioner: Partitioner): JavaPairRDD[K, C] = {
- implicit val cm: ClassManifest[C] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+ implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
fromRDD(rdd.combineByKey(
createCombiner,
mergeValue,
@@ -195,14 +195,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/** Count the number of elements for each key, and return the result to the master as a Map. */
def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey())
- /**
+ /**
* (Experimental) Approximate version of countByKey that can return a partial result if it does
* not finish within a timeout.
*/
def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] =
rdd.countByKeyApprox(timeout).map(mapAsJavaMap)
- /**
+ /**
* (Experimental) Approximate version of countByKey that can return a partial result if it does
* not finish within a timeout.
*/
@@ -258,7 +258,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/**
* Return an RDD with the elements from `this` that are not in `other`.
- *
+ *
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
@@ -315,15 +315,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)})
}
- /**
+ /**
* Simplified version of combineByKey that hash-partitions the resulting RDD using the existing
* partitioner/parallelism level.
*/
def combineByKey[C](createCombiner: JFunction[V, C],
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = {
- implicit val cm: ClassManifest[C] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+ implicit val cm: ClassTag[C] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(rdd)))
}
@@ -414,8 +413,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* this also retains the original RDD's partitioning.
*/
def mapValues[U](f: JFunction[V, U]): JavaPairRDD[K, U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
fromRDD(rdd.mapValues(f))
}
@@ -426,8 +424,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = {
import scala.collection.JavaConverters._
def fn = (x: V) => f.apply(x).asScala
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
fromRDD(rdd.flatMapValues(fn))
}
@@ -592,6 +589,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
}
/**
+ * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling
+ * `collect` or `save` on the resulting RDD will return or output an ordered list of records
+ * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in
+ * order of the keys).
+ */
+ def sortByKey(comp: Comparator[K], ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = {
+ class KeyOrdering(val a: K) extends Ordered[K] {
+ override def compare(b: K) = comp.compare(a, b)
+ }
+ implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x)
+ fromRDD(new OrderedRDDFunctions[K, V, (K, V)](rdd).sortByKey(ascending, numPartitions))
+ }
+
+ /**
* Return an RDD with the keys of each tuple.
*/
def keys(): JavaRDD[K] = JavaRDD.fromRDD[K](rdd.map(_._1))
@@ -600,25 +611,61 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* Return an RDD with the values of each tuple.
*/
def values(): JavaRDD[V] = JavaRDD.fromRDD[V](rdd.map(_._2))
+
+ /**
+ * Return approximate number of distinct values for each key in this RDD.
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. Uses the provided
+ * Partitioner to partition the output RDD.
+ */
+ def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): JavaRDD[(K, Long)] = {
+ rdd.countApproxDistinctByKey(relativeSD, partitioner)
+ }
+
+ /**
+ * Return approximate number of distinct values for each key this RDD.
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. The default value of
+ * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism
+ * level.
+ */
+ def countApproxDistinctByKey(relativeSD: Double = 0.05): JavaRDD[(K, Long)] = {
+ rdd.countApproxDistinctByKey(relativeSD)
+ }
+
+
+ /**
+ * Return approximate number of distinct values for each key in this RDD.
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. HashPartitions the
+ * output RDD into numPartitions.
+ *
+ */
+ def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): JavaRDD[(K, Long)] = {
+ rdd.countApproxDistinctByKey(relativeSD, numPartitions)
+ }
}
object JavaPairRDD {
- def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassManifest[K],
- vcm: ClassManifest[T]): RDD[(K, JList[T])] =
+ def groupByResultToJava[K, T](rdd: RDD[(K, Seq[T])])(implicit kcm: ClassTag[K],
+ vcm: ClassTag[T]): RDD[(K, JList[T])] =
rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList _)
- def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassManifest[K],
- vcm: ClassManifest[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd).mapValues((x: (Seq[V],
- Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2)))
+ def cogroupResultToJava[W, K, V](rdd: RDD[(K, (Seq[V], Seq[W]))])(implicit kcm: ClassTag[K],
+ vcm: ClassTag[V]): RDD[(K, (JList[V], JList[W]))] = rddToPairRDDFunctions(rdd)
+ .mapValues((x: (Seq[V], Seq[W])) => (seqAsJavaList(x._1), seqAsJavaList(x._2)))
def cogroupResult2ToJava[W1, W2, K, V](rdd: RDD[(K, (Seq[V], Seq[W1],
- Seq[W2]))])(implicit kcm: ClassManifest[K]) : RDD[(K, (JList[V], JList[W1],
+ Seq[W2]))])(implicit kcm: ClassTag[K]) : RDD[(K, (JList[V], JList[W1],
JList[W2]))] = rddToPairRDDFunctions(rdd).mapValues(
(x: (Seq[V], Seq[W1], Seq[W2])) => (seqAsJavaList(x._1),
seqAsJavaList(x._2),
seqAsJavaList(x._3)))
- def fromRDD[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]): JavaPairRDD[K, V] =
+ def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd)
implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd
@@ -626,10 +673,8 @@ object JavaPairRDD {
/** Convert a JavaRDD of key-value pairs to JavaPairRDD. */
def fromJavaRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] = {
- implicit val cmk: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val cmv: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cmk: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val cmv: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
new JavaPairRDD[K, V](rdd.rdd)
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
index 3b359a8fd6..037cd1c774 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala
@@ -17,12 +17,14 @@
package org.apache.spark.api.java
+import scala.reflect.ClassTag
+
import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.storage.StorageLevel
-class JavaRDD[T](val rdd: RDD[T])(implicit val classManifest: ClassManifest[T]) extends
+class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends
JavaRDDLike[T, JavaRDD[T]] {
override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd)
@@ -123,12 +125,13 @@ JavaRDDLike[T, JavaRDD[T]] {
*/
def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] =
wrapRDD(rdd.subtract(other, p))
+
+ override def toString = rdd.toString
}
object JavaRDD {
- implicit def fromRDD[T: ClassManifest](rdd: RDD[T]): JavaRDD[T] = new JavaRDD[T](rdd)
+ implicit def fromRDD[T: ClassTag](rdd: RDD[T]): JavaRDD[T] = new JavaRDD[T](rdd)
implicit def toRDD[T](rdd: JavaRDD[T]): RDD[T] = rdd.rdd
}
-
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 7a3568c5ef..924d8af060 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
@@ -20,6 +20,7 @@ package org.apache.spark.api.java
import java.util.{List => JList, Comparator}
import scala.Tuple2
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import com.google.common.base.Optional
import org.apache.hadoop.io.compress.CompressionCodec
@@ -35,7 +36,7 @@ import org.apache.spark.storage.StorageLevel
trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def wrapRDD(rdd: RDD[T]): This
- implicit val classManifest: ClassManifest[T]
+ implicit val classTag: ClassTag[T]
def rdd: RDD[T]
@@ -71,7 +72,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* Return a new RDD by applying a function to each partition of this RDD, while tracking the index
* of the original partition.
*/
- def mapPartitionsWithIndex[R: ClassManifest](
+ def mapPartitionsWithIndex[R: ClassTag](
f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]],
preservesPartitioning: Boolean = false): JavaRDD[R] =
new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))),
@@ -87,7 +88,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* Return a new RDD by applying a function to all elements of this RDD.
*/
def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = {
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
new JavaPairRDD(rdd.map(f)(cm))(f.keyType(), f.valueType())
}
@@ -118,7 +119,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = {
import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType())
}
@@ -158,18 +159,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* elements (a, b) where a is in `this` and b is in `other`.
*/
def cartesian[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] =
- JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest,
- other.classManifest)
+ JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classTag))(classTag, other.classTag)
/**
* Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
* mapping to that key.
*/
def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = {
- implicit val kcm: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val vcm: ClassManifest[JList[T]] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]]
+ implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val vcm: ClassTag[JList[T]] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]]
JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm)
}
@@ -178,10 +177,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* mapping to that key.
*/
def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = {
- implicit val kcm: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val vcm: ClassManifest[JList[T]] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]]
+ implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val vcm: ClassTag[JList[T]] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[JList[T]]]
JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm)
}
@@ -209,7 +207,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* a map on the other).
*/
def zip[U](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] = {
- JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classManifest))(classManifest, other.classManifest)
+ JavaPairRDD.fromRDD(rdd.zip(other.rdd)(other.classTag))(classTag, other.classTag)
}
/**
@@ -224,7 +222,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator(
f.apply(asJavaIterator(x), asJavaIterator(y)).iterator())
JavaRDD.fromRDD(
- rdd.zipPartitions(other.rdd)(fn)(other.classManifest, f.elementType()))(f.elementType())
+ rdd.zipPartitions(other.rdd)(fn)(other.classTag, f.elementType()))(f.elementType())
}
// Actions (launch a job to return a value to the user program)
@@ -247,6 +245,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
}
/**
+ * Return an array that contains all of the elements in a specific partition of this RDD.
+ */
+ def collectPartitions(partitionIds: Array[Int]): Array[JList[T]] = {
+ // This is useful for implementing `take` from other language frontends
+ // like Python where the data is serialized.
+ import scala.collection.JavaConversions._
+ val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds, true)
+ res.map(x => new java.util.ArrayList(x.toSeq)).toArray
+ }
+
+ /**
* Reduces the elements of this RDD using the specified commutative and associative binary operator.
*/
def reduce(f: JFunction2[T, T, T]): T = rdd.reduce(f)
@@ -356,7 +365,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* Creates tuples of the elements in this RDD by applying `f`.
*/
def keyBy[K](f: JFunction[T, K]): JavaPairRDD[K, T] = {
- implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
+ implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
JavaPairRDD.fromRDD(rdd.keyBy(f))
}
@@ -435,4 +444,15 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[T]]
takeOrdered(num, comp)
}
+
+ /**
+ * Return approximate number of distinct elements in the RDD.
+ *
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. The default value of
+ * relativeSD is 0.05.
+ */
+ def countApproxDistinct(relativeSD: Double = 0.05): Long = rdd.countApproxDistinct(relativeSD)
+
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 8869e072bf..e93b10fd7e 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -21,6 +21,7 @@ import java.util.{Map => JMap}
import scala.collection.JavaConversions
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.InputFormat
@@ -28,17 +29,22 @@ import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import com.google.common.base.Optional
-import org.apache.spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, SparkContext}
+import org.apache.spark._
import org.apache.spark.SparkContext.IntAccumulatorParam
import org.apache.spark.SparkContext.DoubleAccumulatorParam
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
+import scala.Tuple2
/**
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and
* works with Java collections instead of Scala ones.
*/
class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
+ /**
+ * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters
+ */
+ def this(conf: SparkConf) = this(new SparkContext(conf))
/**
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
@@ -49,6 +55,14 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
/**
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
* @param appName A name for your application, to display on the cluster web UI
+ * @param conf a [[org.apache.spark.SparkConf]] object specifying other Spark parameters
+ */
+ def this(master: String, appName: String, conf: SparkConf) =
+ this(conf.setMaster(master).setAppName(appName))
+
+ /**
+ * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+ * @param appName A name for your application, to display on the cluster web UI
* @param sparkHome The SPARK_HOME directory on the slave nodes
* @param jarFile JAR file to send to the cluster. This can be a path on the local file system
* or an HDFS, HTTP, HTTPS, or FTP URL.
@@ -82,8 +96,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
/** Distribute a local Scala collection to form an RDD. */
def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)
}
@@ -94,10 +107,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
/** Distribute a local Scala collection to form an RDD. */
def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]], numSlices: Int)
: JavaPairRDD[K, V] = {
- implicit val kcm: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val vcm: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val kcm: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val vcm: ClassTag[V] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices))
}
@@ -132,16 +143,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
valueClass: Class[V],
minSplits: Int
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minSplits))
}
/**Get an RDD for a Hadoop SequenceFile. */
def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]):
JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass))
}
@@ -153,8 +164,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
* that there's very little effort required to save arbitrary objects.
*/
def objectFile[T](path: String, minSplits: Int): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
sc.objectFile(path, minSplits)(cm)
}
@@ -166,8 +176,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
* that there's very little effort required to save arbitrary objects.
*/
def objectFile[T](path: String): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
sc.objectFile(path)(cm)
}
@@ -183,8 +192,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
valueClass: Class[V],
minSplits: Int
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits))
}
@@ -199,8 +208,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
keyClass: Class[K],
valueClass: Class[V]
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass))
}
@@ -212,8 +221,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
valueClass: Class[V],
minSplits: Int
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits))
}
@@ -224,8 +233,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
keyClass: Class[K],
valueClass: Class[V]
): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(keyClass)
- implicit val vcm = ClassManifest.fromClass(valueClass)
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
new JavaPairRDD(sc.hadoopFile(path,
inputFormatClass, keyClass, valueClass))
}
@@ -240,8 +249,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
kClass: Class[K],
vClass: Class[V],
conf: Configuration): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(kClass)
- implicit val vcm = ClassManifest.fromClass(vClass)
+ implicit val kcm: ClassTag[K] = ClassTag(kClass)
+ implicit val vcm: ClassTag[V] = ClassTag(vClass)
new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf))
}
@@ -254,15 +263,15 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
fClass: Class[F],
kClass: Class[K],
vClass: Class[V]): JavaPairRDD[K, V] = {
- implicit val kcm = ClassManifest.fromClass(kClass)
- implicit val vcm = ClassManifest.fromClass(vClass)
+ implicit val kcm: ClassTag[K] = ClassTag(kClass)
+ implicit val vcm: ClassTag[V] = ClassTag(vClass)
new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass))
}
/** Build the union of two or more RDDs. */
override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = {
val rdds: Seq[RDD[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd)
- implicit val cm: ClassManifest[T] = first.classManifest
+ implicit val cm: ClassTag[T] = first.classTag
sc.union(rdds)(cm)
}
@@ -270,9 +279,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
override def union[K, V](first: JavaPairRDD[K, V], rest: java.util.List[JavaPairRDD[K, V]])
: JavaPairRDD[K, V] = {
val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd)
- implicit val cm: ClassManifest[(K, V)] = first.classManifest
- implicit val kcm: ClassManifest[K] = first.kManifest
- implicit val vcm: ClassManifest[V] = first.vManifest
+ implicit val cm: ClassTag[(K, V)] = first.classTag
+ implicit val kcm: ClassTag[K] = first.kClassTag
+ implicit val vcm: ClassTag[V] = first.vClassTag
new JavaPairRDD(sc.union(rdds)(cm))(kcm, vcm)
}
@@ -385,34 +394,47 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
/**
* Set the directory under which RDDs are going to be checkpointed. The directory must
- * be a HDFS path if running on a cluster. If the directory does not exist, it will
- * be created. If the directory exists and useExisting is set to true, then the
- * exisiting directory will be used. Otherwise an exception will be thrown to
- * prevent accidental overriding of checkpoint files in the existing directory.
- */
- def setCheckpointDir(dir: String, useExisting: Boolean) {
- sc.setCheckpointDir(dir, useExisting)
- }
-
- /**
- * Set the directory under which RDDs are going to be checkpointed. The directory must
- * be a HDFS path if running on a cluster. If the directory does not exist, it will
- * be created. If the directory exists, an exception will be thrown to prevent accidental
- * overriding of checkpoint files.
+ * be a HDFS path if running on a cluster.
*/
def setCheckpointDir(dir: String) {
sc.setCheckpointDir(dir)
}
protected def checkpointFile[T](path: String): JavaRDD[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
new JavaRDD(sc.checkpointFile(path))
}
+
+ /**
+ * Return a copy of this JavaSparkContext's configuration. The configuration ''cannot'' be
+ * changed at runtime.
+ */
+ def getConf: SparkConf = sc.getConf
+
+ /**
+ * Pass-through to SparkContext.setCallSite. For API support only.
+ */
+ def setCallSite(site: String) {
+ sc.setCallSite(site)
+ }
+
+ /**
+ * Pass-through to SparkContext.setCallSite. For API support only.
+ */
+ def clearCallSite() {
+ sc.clearCallSite()
+ }
}
object JavaSparkContext {
implicit def fromSparkContext(sc: SparkContext): JavaSparkContext = new JavaSparkContext(sc)
implicit def toSparkContext(jsc: JavaSparkContext): SparkContext = jsc.sc
+
+ /**
+ * Find the JAR from which a given class was loaded, to make it easy for users to pass
+ * their JARs to SparkContext.
+ */
+ def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
index c9cbce5624..2090efd3b9 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java
@@ -17,7 +17,6 @@
package org.apache.spark.api.java;
-import java.util.Arrays;
import java.util.ArrayList;
import java.util.List;
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
index 2dfda8b09a..bdb01f7670 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala
@@ -17,9 +17,11 @@
package org.apache.spark.api.java.function
+import scala.reflect.ClassTag
+
/**
* A function that returns zero or more output records from each input record.
*/
abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]] {
- def elementType() : ClassManifest[R] = ClassManifest.Any.asInstanceOf[ClassManifest[R]]
+ def elementType(): ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]]
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
index 528e1c0a7c..aae1349c5e 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala
@@ -17,9 +17,11 @@
package org.apache.spark.api.java.function
+import scala.reflect.ClassTag
+
/**
* A function that takes two inputs and returns zero or more output records.
*/
abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Iterable[C]] {
- def elementType() : ClassManifest[C] = ClassManifest.Any.asInstanceOf[ClassManifest[C]]
+ def elementType() : ClassTag[C] = ClassTag.Any.asInstanceOf[ClassTag[C]]
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.java b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
index ce368ee01b..537439ef53 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function.java
@@ -17,8 +17,8 @@
package org.apache.spark.api.java.function;
-import scala.reflect.ClassManifest;
-import scala.reflect.ClassManifest$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
import java.io.Serializable;
@@ -29,8 +29,8 @@ import java.io.Serializable;
* when mapping RDDs of other types.
*/
public abstract class Function<T, R> extends WrappedFunction1<T, R> implements Serializable {
- public ClassManifest<R> returnType() {
- return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<R> returnType() {
+ return ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
index 44ad559d48..a2d1214fb4 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function2.java
@@ -17,8 +17,8 @@
package org.apache.spark.api.java.function;
-import scala.reflect.ClassManifest;
-import scala.reflect.ClassManifest$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
import java.io.Serializable;
@@ -28,8 +28,8 @@ import java.io.Serializable;
public abstract class Function2<T1, T2, R> extends WrappedFunction2<T1, T2, R>
implements Serializable {
- public ClassManifest<R> returnType() {
- return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<R> returnType() {
+ return (ClassTag<R>) ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
index ac6178924a..fb1deceab5 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/Function3.java
@@ -17,8 +17,8 @@
package org.apache.spark.api.java.function;
-import scala.reflect.ClassManifest;
-import scala.reflect.ClassManifest$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
import scala.runtime.AbstractFunction2;
import java.io.Serializable;
@@ -29,8 +29,8 @@ import java.io.Serializable;
public abstract class Function3<T1, T2, T3, R> extends WrappedFunction3<T1, T2, T3, R>
implements Serializable {
- public ClassManifest<R> returnType() {
- return (ClassManifest<R>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<R> returnType() {
+ return (ClassTag<R>) ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
index 6d76a8f970..ca485b3cc2 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.java
@@ -18,8 +18,8 @@
package org.apache.spark.api.java.function;
import scala.Tuple2;
-import scala.reflect.ClassManifest;
-import scala.reflect.ClassManifest$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
import java.io.Serializable;
@@ -33,11 +33,11 @@ public abstract class PairFlatMapFunction<T, K, V>
extends WrappedFunction1<T, Iterable<Tuple2<K, V>>>
implements Serializable {
- public ClassManifest<K> keyType() {
- return (ClassManifest<K>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<K> keyType() {
+ return (ClassTag<K>) ClassTag$.MODULE$.apply(Object.class);
}
- public ClassManifest<V> valueType() {
- return (ClassManifest<V>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<V> valueType() {
+ return (ClassTag<V>) ClassTag$.MODULE$.apply(Object.class);
}
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
index ede7ceefb5..cbe2306026 100644
--- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
+++ b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.java
@@ -18,8 +18,8 @@
package org.apache.spark.api.java.function;
import scala.Tuple2;
-import scala.reflect.ClassManifest;
-import scala.reflect.ClassManifest$;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
import java.io.Serializable;
@@ -31,11 +31,11 @@ import java.io.Serializable;
public abstract class PairFunction<T, K, V> extends WrappedFunction1<T, Tuple2<K, V>>
implements Serializable {
- public ClassManifest<K> keyType() {
- return (ClassManifest<K>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<K> keyType() {
+ return (ClassTag<K>) ClassTag$.MODULE$.apply(Object.class);
}
- public ClassManifest<V> valueType() {
- return (ClassManifest<V>) ClassManifest$.MODULE$.fromClass(Object.class);
+ public ClassTag<V> valueType() {
+ return (ClassTag<V>) ClassTag$.MODULE$.apply(Object.class);
}
}
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 12b4d94a56..32cc70e8c9 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
@@ -22,18 +22,17 @@ import java.net._
import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD}
import org.apache.spark.broadcast.Broadcast
import org.apache.spark._
import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.PipedRDD
import org.apache.spark.util.Utils
-
-private[spark] class PythonRDD[T: ClassManifest](
+private[spark] class PythonRDD[T: ClassTag](
parent: RDD[T],
- command: Seq[String],
+ command: Array[Byte],
envVars: JMap[String, String],
pythonIncludes: JList[String],
preservePartitoning: Boolean,
@@ -42,23 +41,12 @@ private[spark] class PythonRDD[T: ClassManifest](
accumulator: Accumulator[JList[Array[Byte]]])
extends RDD[Array[Byte]](parent) {
- val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
-
- // Similar to Runtime.exec(), if we are given a single string, split it into words
- // using a standard StringTokenizer (i.e. by spaces)
- def this(parent: RDD[T], command: String, envVars: JMap[String, String],
- pythonIncludes: JList[String],
- preservePartitoning: Boolean, pythonExec: String,
- broadcastVars: JList[Broadcast[Array[Byte]]],
- accumulator: Accumulator[JList[Array[Byte]]]) =
- this(parent, PipedRDD.tokenize(command), envVars, pythonIncludes, preservePartitoning, pythonExec,
- broadcastVars, accumulator)
+ val bufferSize = conf.get("spark.buffer.size", "65536").toInt
override def getPartitions = parent.partitions
override val partitioner = if (preservePartitoning) parent.partitioner else None
-
override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
val startTime = System.currentTimeMillis
val env = SparkEnv.get
@@ -71,11 +59,10 @@ private[spark] class PythonRDD[T: ClassManifest](
SparkEnv.set(env)
val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize)
val dataOut = new DataOutputStream(stream)
- val printOut = new PrintWriter(stream)
// Partition index
dataOut.writeInt(split.index)
// sparkFilesDir
- PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dataOut)
+ dataOut.writeUTF(SparkFiles.getRootDirectory)
// Broadcast variables
dataOut.writeInt(broadcastVars.length)
for (broadcast <- broadcastVars) {
@@ -85,21 +72,16 @@ private[spark] class PythonRDD[T: ClassManifest](
}
// Python includes (*.zip and *.egg files)
dataOut.writeInt(pythonIncludes.length)
- for (f <- pythonIncludes) {
- PythonRDD.writeAsPickle(f, dataOut)
- }
+ pythonIncludes.foreach(dataOut.writeUTF)
dataOut.flush()
- // Serialized user code
- for (elem <- command) {
- printOut.println(elem)
- }
- printOut.flush()
+ // Serialized command:
+ dataOut.writeInt(command.length)
+ dataOut.write(command)
// Data values
for (elem <- parent.iterator(split, context)) {
- PythonRDD.writeAsPickle(elem, dataOut)
+ PythonRDD.writeToStream(elem, dataOut)
}
dataOut.flush()
- printOut.flush()
worker.shutdownOutput()
} catch {
case e: IOException =>
@@ -132,7 +114,7 @@ private[spark] class PythonRDD[T: ClassManifest](
val obj = new Array[Byte](length)
stream.readFully(obj)
obj
- case -3 =>
+ case SpecialLengths.TIMING_DATA =>
// Timing data from worker
val bootTime = stream.readLong()
val initTime = stream.readLong()
@@ -143,30 +125,30 @@ private[spark] class PythonRDD[T: ClassManifest](
val total = finishTime - startTime
logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish))
read
- case -2 =>
+ case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
// Signals that an exception has been thrown in python
val exLength = stream.readInt()
val obj = new Array[Byte](exLength)
stream.readFully(obj)
throw new PythonException(new String(obj))
- case -1 =>
+ case SpecialLengths.END_OF_DATA_SECTION =>
// We've finished the data section of the output, but we can still
- // read some accumulator updates; let's do that, breaking when we
- // get a negative length record.
- var len2 = stream.readInt()
- while (len2 >= 0) {
- val update = new Array[Byte](len2)
+ // read some accumulator updates:
+ val numAccumulatorUpdates = stream.readInt()
+ (1 to numAccumulatorUpdates).foreach { _ =>
+ val updateLen = stream.readInt()
+ val update = new Array[Byte](updateLen)
stream.readFully(update)
accumulator += Collections.singletonList(update)
- len2 = stream.readInt()
+
}
- new Array[Byte](0)
+ Array.empty[Byte]
}
} catch {
case eof: EOFException => {
throw new SparkException("Python worker exited unexpectedly (crashed)", eof)
}
- case e => throw e
+ case e: Throwable => throw e
}
}
@@ -197,62 +179,15 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
val asJavaPairRDD : JavaPairRDD[Long, Array[Byte]] = JavaPairRDD.fromRDD(this)
}
-private[spark] object PythonRDD {
-
- /** Strips the pickle PROTO and STOP opcodes from the start and end of a pickle */
- def stripPickle(arr: Array[Byte]) : Array[Byte] = {
- arr.slice(2, arr.length - 1)
- }
+private object SpecialLengths {
+ val END_OF_DATA_SECTION = -1
+ val PYTHON_EXCEPTION_THROWN = -2
+ val TIMING_DATA = -3
+}
- /**
- * Write strings, pickled Python objects, or pairs of pickled objects to a data output stream.
- * The data format is a 32-bit integer representing the pickled object's length (in bytes),
- * followed by the pickled data.
- *
- * Pickle module:
- *
- * http://docs.python.org/2/library/pickle.html
- *
- * The pickle protocol is documented in the source of the `pickle` and `pickletools` modules:
- *
- * http://hg.python.org/cpython/file/2.6/Lib/pickle.py
- * http://hg.python.org/cpython/file/2.6/Lib/pickletools.py
- *
- * @param elem the object to write
- * @param dOut a data output stream
- */
- def writeAsPickle(elem: Any, dOut: DataOutputStream) {
- if (elem.isInstanceOf[Array[Byte]]) {
- val arr = elem.asInstanceOf[Array[Byte]]
- dOut.writeInt(arr.length)
- dOut.write(arr)
- } else if (elem.isInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]) {
- val t = elem.asInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]
- val length = t._1.length + t._2.length - 3 - 3 + 4 // stripPickle() removes 3 bytes
- dOut.writeInt(length)
- dOut.writeByte(Pickle.PROTO)
- dOut.writeByte(Pickle.TWO)
- dOut.write(PythonRDD.stripPickle(t._1))
- dOut.write(PythonRDD.stripPickle(t._2))
- dOut.writeByte(Pickle.TUPLE2)
- dOut.writeByte(Pickle.STOP)
- } else if (elem.isInstanceOf[String]) {
- // For uniformity, strings are wrapped into Pickles.
- val s = elem.asInstanceOf[String].getBytes("UTF-8")
- val length = 2 + 1 + 4 + s.length + 1
- dOut.writeInt(length)
- dOut.writeByte(Pickle.PROTO)
- dOut.writeByte(Pickle.TWO)
- dOut.write(Pickle.BINUNICODE)
- dOut.writeInt(Integer.reverseBytes(s.length))
- dOut.write(s)
- dOut.writeByte(Pickle.STOP)
- } else {
- throw new SparkException("Unexpected RDD type")
- }
- }
+private[spark] object PythonRDD {
- def readRDDFromPickleFile(sc: JavaSparkContext, filename: String, parallelism: Int) :
+ def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int):
JavaRDD[Array[Byte]] = {
val file = new DataInputStream(new FileInputStream(filename))
val objs = new collection.mutable.ArrayBuffer[Array[Byte]]
@@ -265,39 +200,41 @@ private[spark] object PythonRDD {
}
} catch {
case eof: EOFException => {}
- case e => throw e
+ case e: Throwable => throw e
}
JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism))
}
- def writeIteratorToPickleFile[T](items: java.util.Iterator[T], filename: String) {
+ def writeToStream(elem: Any, dataOut: DataOutputStream) {
+ elem match {
+ case bytes: Array[Byte] =>
+ dataOut.writeInt(bytes.length)
+ dataOut.write(bytes)
+ case pair: (Array[Byte], Array[Byte]) =>
+ dataOut.writeInt(pair._1.length)
+ dataOut.write(pair._1)
+ dataOut.writeInt(pair._2.length)
+ dataOut.write(pair._2)
+ case str: String =>
+ dataOut.writeUTF(str)
+ case other =>
+ throw new SparkException("Unexpected element type " + other.getClass)
+ }
+ }
+
+ def writeToFile[T](items: java.util.Iterator[T], filename: String) {
import scala.collection.JavaConverters._
- writeIteratorToPickleFile(items.asScala, filename)
+ writeToFile(items.asScala, filename)
}
- def writeIteratorToPickleFile[T](items: Iterator[T], filename: String) {
+ def writeToFile[T](items: Iterator[T], filename: String) {
val file = new DataOutputStream(new FileOutputStream(filename))
for (item <- items) {
- writeAsPickle(item, file)
+ writeToStream(item, file)
}
file.close()
}
- def takePartition[T](rdd: RDD[T], partition: Int): Iterator[T] = {
- implicit val cm : ClassManifest[T] = rdd.elementClassManifest
- rdd.context.runJob(rdd, ((x: Iterator[T]) => x.toArray), Seq(partition), true).head.iterator
- }
-}
-
-private object Pickle {
- val PROTO: Byte = 0x80.toByte
- val TWO: Byte = 0x02.toByte
- val BINUNICODE: Byte = 'X'
- val STOP: Byte = '.'
- val TUPLE2: Byte = 0x86.toByte
- val EMPTY_LIST: Byte = ']'
- val MARK: Byte = '('
- val APPENDS: Byte = 'e'
}
private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] {
@@ -313,7 +250,7 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort:
Utils.checkHost(serverHost, "Expected hostname")
- val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+ val bufferSize = SparkEnv.get.conf.get("spark.buffer.size", "65536").toInt
override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
index 67d45723ba..f291266fcf 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala
@@ -64,7 +64,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
startDaemon()
new Socket(daemonHost, daemonPort)
}
- case e => throw e
+ case e: Throwable => throw e
}
}
}
@@ -198,7 +198,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String
}
}.start()
} catch {
- case e => {
+ case e: Throwable => {
stopDaemon()
throw e
}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
index 43c18294c5..0fc478a419 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -31,8 +31,8 @@ abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
override def toString = "Broadcast(" + id + ")"
}
-private[spark]
-class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable {
+private[spark]
+class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable {
private var initialized = false
private var broadcastFactory: BroadcastFactory = null
@@ -43,14 +43,14 @@ class BroadcastManager(val _isDriver: Boolean) extends Logging with Serializable
private def initialize() {
synchronized {
if (!initialized) {
- val broadcastFactoryClass = System.getProperty(
+ val broadcastFactoryClass = conf.get(
"spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory")
broadcastFactory =
Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory]
// Initialize appropriate BroadcastFactory and BroadcastObject
- broadcastFactory.initialize(isDriver)
+ broadcastFactory.initialize(isDriver, conf)
initialized = true
}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
index 68bff75b90..fb161ce69d 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala
@@ -17,6 +17,8 @@
package org.apache.spark.broadcast
+import org.apache.spark.SparkConf
+
/**
* An interface for all the broadcast implementations in Spark (to allow
* multiple broadcast implementations). SparkContext uses a user-specified
@@ -24,7 +26,7 @@ package org.apache.spark.broadcast
* entire Spark job.
*/
private[spark] trait BroadcastFactory {
- def initialize(isDriver: Boolean): Unit
+ def initialize(isDriver: Boolean, conf: SparkConf): Unit
def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T]
def stop(): Unit
}
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 609464e38d..db596d5fcc 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala
@@ -19,18 +19,19 @@ package org.apache.spark.broadcast
import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream}
import java.net.URL
+import java.util.concurrent.TimeUnit
import it.unimi.dsi.fastutil.io.FastBufferedInputStream
import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
-import org.apache.spark.{HttpServer, Logging, SparkEnv}
+import org.apache.spark.{SparkConf, HttpServer, Logging, SparkEnv}
import org.apache.spark.io.CompressionCodec
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 = BroadcastBlockId(id)
@@ -39,7 +40,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea
SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
}
- if (!isLocal) {
+ if (!isLocal) {
HttpBroadcast.write(id, value_)
}
@@ -63,7 +64,7 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea
}
private[spark] class HttpBroadcastFactory extends BroadcastFactory {
- def initialize(isDriver: Boolean) { HttpBroadcast.initialize(isDriver) }
+ def initialize(isDriver: Boolean, conf: SparkConf) { HttpBroadcast.initialize(isDriver, conf) }
def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
new HttpBroadcast[T](value_, isLocal, id)
@@ -80,42 +81,51 @@ private object HttpBroadcast extends Logging {
private var serverUri: String = null
private var server: HttpServer = null
+ // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist
private val files = new TimeStampedHashSet[String]
- private val cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup)
+ private var cleaner: MetadataCleaner = null
+
+ private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt
- private lazy val compressionCodec = CompressionCodec.createCodec()
+ private var compressionCodec: CompressionCodec = null
- def initialize(isDriver: Boolean) {
+ def initialize(isDriver: Boolean, conf: SparkConf) {
synchronized {
if (!initialized) {
- bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
- compress = System.getProperty("spark.broadcast.compress", "true").toBoolean
+ bufferSize = conf.get("spark.buffer.size", "65536").toInt
+ compress = conf.get("spark.broadcast.compress", "true").toBoolean
if (isDriver) {
- createServer()
+ createServer(conf)
+ conf.set("spark.httpBroadcast.uri", serverUri)
}
- serverUri = System.getProperty("spark.httpBroadcast.uri")
+ serverUri = conf.get("spark.httpBroadcast.uri")
+ cleaner = new MetadataCleaner(MetadataCleanerType.HTTP_BROADCAST, cleanup, conf)
+ compressionCodec = CompressionCodec.createCodec(conf)
initialized = true
}
}
}
-
+
def stop() {
synchronized {
if (server != null) {
server.stop()
server = null
}
+ if (cleaner != null) {
+ cleaner.cancel()
+ cleaner = null
+ }
+ compressionCodec = null
initialized = false
- cleaner.cancel()
}
}
- private def createServer() {
- broadcastDir = Utils.createTempDir(Utils.getLocalDir)
+ private def createServer(conf: SparkConf) {
+ broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf))
server = new HttpServer(broadcastDir)
server.start()
serverUri = server.uri
- System.setProperty("spark.httpBroadcast.uri", serverUri)
logInfo("Broadcast server started at " + serverUri)
}
@@ -138,10 +148,13 @@ private object HttpBroadcast extends Logging {
def read[T](id: Long): T = {
val url = serverUri + "/" + BroadcastBlockId(id).name
val in = {
+ val httpConnection = new URL(url).openConnection()
+ httpConnection.setReadTimeout(httpReadTimeout)
+ val inputStream = httpConnection.getInputStream
if (compress) {
- compressionCodec.compressedInputStream(new URL(url).openStream())
+ compressionCodec.compressedInputStream(inputStream)
} else {
- new FastBufferedInputStream(new URL(url).openStream(), bufferSize)
+ new FastBufferedInputStream(inputStream, bufferSize)
}
}
val ser = SparkEnv.get.serializer.newInstance()
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index 073a0a5029..9530938278 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -83,13 +83,13 @@ extends Broadcast[T](id) with Logging with Serializable {
case None =>
val start = System.nanoTime
logInfo("Started reading broadcast variable " + id)
-
+
// Initialize @transient variables that will receive garbage values from the master.
resetWorkerVariables()
if (receiveBroadcast(id)) {
value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
-
+
// Store the merged copy in cache so that the next worker doesn't need to rebuild it.
// This creates a tradeoff between memory usage and latency.
// Storing copy doubles the memory footprint; not storing doubles deserialization cost.
@@ -122,14 +122,14 @@ extends Broadcast[T](id) with Logging with Serializable {
while (attemptId > 0 && totalBlocks == -1) {
TorrentBroadcast.synchronized {
SparkEnv.get.blockManager.getSingle(metaId) match {
- case Some(x) =>
+ case Some(x) =>
val tInfo = x.asInstanceOf[TorrentInfo]
totalBlocks = tInfo.totalBlocks
totalBytes = tInfo.totalBytes
arrayOfBlocks = new Array[TorrentBlock](totalBlocks)
hasBlocks = 0
-
- case None =>
+
+ case None =>
Thread.sleep(500)
}
}
@@ -145,13 +145,13 @@ extends Broadcast[T](id) with Logging with Serializable {
val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid)
TorrentBroadcast.synchronized {
SparkEnv.get.blockManager.getSingle(pieceId) match {
- case Some(x) =>
+ case Some(x) =>
arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock]
hasBlocks += 1
SparkEnv.get.blockManager.putSingle(
pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true)
-
- case None =>
+
+ case None =>
throw new SparkException("Failed to get " + pieceId + " of " + broadcastId)
}
}
@@ -166,21 +166,22 @@ private object TorrentBroadcast
extends Logging {
private var initialized = false
-
- def initialize(_isDriver: Boolean) {
+ private var conf: SparkConf = null
+ def initialize(_isDriver: Boolean, conf: SparkConf) {
+ TorrentBroadcast.conf = conf //TODO: we might have to fix it in tests
synchronized {
if (!initialized) {
initialized = true
}
}
}
-
+
def stop() {
initialized = false
}
- val BLOCK_SIZE = System.getProperty("spark.broadcast.blockSize", "4096").toInt * 1024
-
+ lazy val BLOCK_SIZE = conf.get("spark.broadcast.blockSize", "4096").toInt * 1024
+
def blockifyObject[T](obj: T): TorrentInfo = {
val byteArray = Utils.serialize[T](obj)
val bais = new ByteArrayInputStream(byteArray)
@@ -209,7 +210,7 @@ extends Logging {
}
def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock],
- totalBytes: Int,
+ totalBytes: Int,
totalBlocks: Int): T = {
var retByteArray = new Array[Byte](totalBytes)
for (i <- 0 until totalBlocks) {
@@ -222,23 +223,23 @@ extends Logging {
}
private[spark] case class TorrentBlock(
- blockID: Int,
- byteArray: Array[Byte])
+ blockID: Int,
+ byteArray: Array[Byte])
extends Serializable
private[spark] case class TorrentInfo(
@transient arrayOfBlocks : Array[TorrentBlock],
- totalBlocks: Int,
- totalBytes: Int)
+ totalBlocks: Int,
+ totalBytes: Int)
extends Serializable {
-
- @transient var hasBlocks = 0
+
+ @transient var hasBlocks = 0
}
private[spark] class TorrentBroadcastFactory
extends BroadcastFactory {
-
- def initialize(isDriver: Boolean) { TorrentBroadcast.initialize(isDriver) }
+
+ def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) }
def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
new TorrentBroadcast[T](value_, isLocal, id)
diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
index 19d393a0db..e38459b883 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
@@ -19,7 +19,7 @@ package org.apache.spark.deploy
private[spark] class ApplicationDescription(
val name: String,
- val maxCores: Int, /* Integer.MAX_VALUE denotes an unlimited number of cores */
+ val maxCores: Option[Int],
val memoryPerSlave: Int,
val command: Command,
val sparkHome: String,
diff --git a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
index fcfea96ad6..37dfa7fec0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ExecutorState.scala
@@ -17,8 +17,7 @@
package org.apache.spark.deploy
-private[spark] object ExecutorState
- extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") {
+private[spark] object ExecutorState extends Enumeration {
val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value
diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
index 0aa8852649..4dfb19ed8a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala
@@ -190,7 +190,7 @@ private[spark] object FaultToleranceTest extends App with Logging {
/** 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
+ // Counter-hack: Because of a hack in SparkEnv#create() that changes this
// property, we need to reset it.
System.setProperty("spark.driver.port", "0")
sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome)
@@ -417,4 +417,4 @@ private[spark] object Docker extends Logging {
"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/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index a724900943..ffc0cb0903 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -22,7 +22,7 @@ import akka.actor.ActorSystem
import org.apache.spark.deploy.worker.Worker
import org.apache.spark.deploy.master.Master
import org.apache.spark.util.Utils
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
import scala.collection.mutable.ArrayBuffer
@@ -34,16 +34,17 @@ import scala.collection.mutable.ArrayBuffer
*/
private[spark]
class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: Int) extends Logging {
-
+
private val localHostname = Utils.localHostName()
private val masterActorSystems = ArrayBuffer[ActorSystem]()
private val workerActorSystems = ArrayBuffer[ActorSystem]()
-
+
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 conf = new SparkConf(false)
+ val (masterSystem, masterPort, _) = Master.startSystemAndActor(localHostname, 0, 0, conf)
masterActorSystems += masterSystem
val masterUrl = "spark://" + localHostname + ":" + masterPort
val masters = Array(masterUrl)
@@ -55,16 +56,19 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I
workerActorSystems += workerSystem
}
- return masters
+ masters
}
def stop() {
logInfo("Shutting down local Spark cluster.")
// Stop the workers before the master so they don't get upset that it disconnected
+ // TODO: In Akka 2.1.x, ActorSystem.awaitTermination hangs when you have remote actors!
+ // This is unfortunate, but for now we just comment it out.
workerActorSystems.foreach(_.shutdown())
- workerActorSystems.foreach(_.awaitTermination())
-
+ //workerActorSystems.foreach(_.awaitTermination())
masterActorSystems.foreach(_.shutdown())
- masterActorSystems.foreach(_.awaitTermination())
+ //masterActorSystems.foreach(_.awaitTermination())
+ masterActorSystems.clear()
+ workerActorSystems.clear()
}
}
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 fc1537f796..27dc42bf7e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -34,10 +34,10 @@ class SparkHadoopUtil {
UserGroupInformation.setConfiguration(conf)
def runAsUser(user: String)(func: () => Unit) {
- // if we are already running as the user intended there is no reason to do the doAs. It
+ // if we are already running as the user intended there is no reason to do the doAs. It
// will actually break secure HDFS access as it doesn't fill in the credentials. Also if
- // the user is UNKNOWN then we shouldn't be creating a remote unknown user
- // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only
+ // the user is UNKNOWN then we shouldn't be creating a remote unknown user
+ // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only
// in SparkContext.
val currentUser = Option(System.getProperty("user.name")).
getOrElse(SparkContext.SPARK_UNKNOWN_USER)
@@ -67,11 +67,15 @@ class SparkHadoopUtil {
}
object SparkHadoopUtil {
+
private val hadoop = {
- val yarnMode = java.lang.Boolean.valueOf(System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
+ val yarnMode = java.lang.Boolean.valueOf(
+ System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
if (yarnMode) {
try {
- Class.forName("org.apache.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/deploy/client/Client.scala b/core/src/main/scala/org/apache/spark/deploy/client/Client.scala
index 77422f61ec..481026eaa2 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
@@ -19,21 +19,18 @@ package org.apache.spark.deploy.client
import java.util.concurrent.TimeoutException
+import scala.concurrent.Await
+import scala.concurrent.duration._
+
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
-import akka.dispatch.Await
-
-import org.apache.spark.Logging
+import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent}
+
+import org.apache.spark.{Logging, SparkConf, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.Master
-
+import org.apache.spark.util.AkkaUtils
/**
* The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
@@ -45,24 +42,26 @@ private[spark] class Client(
actorSystem: ActorSystem,
masterUrls: Array[String],
appDescription: ApplicationDescription,
- listener: ClientListener)
+ listener: ClientListener,
+ conf: SparkConf)
extends Logging {
val REGISTRATION_TIMEOUT = 20.seconds
val REGISTRATION_RETRIES = 3
+ var masterAddress: Address = null
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 master: ActorSelection = null
var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times
var alreadyDead = false // To avoid calling listener.dead() multiple times
override def preStart() {
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
try {
registerWithMaster()
} catch {
@@ -76,7 +75,7 @@ private[spark] class Client(
def tryRegisterAllMasters() {
for (masterUrl <- masterUrls) {
logInfo("Connecting to master " + masterUrl + "...")
- val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+ val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
actor ! RegisterApplication(appDescription)
}
}
@@ -84,6 +83,7 @@ private[spark] class Client(
def registerWithMaster() {
tryRegisterAllMasters()
+ import context.dispatcher
var retries = 0
lazy val retryTimer: Cancellable =
context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) {
@@ -102,10 +102,19 @@ private[spark] class Client(
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
+ master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
+ masterAddress = activeMasterUrl match {
+ case Master.sparkUrlRegex(host, port) =>
+ Address("akka.tcp", Master.systemName, host, port.toInt)
+ case x =>
+ throw new SparkException("Invalid spark URL: " + x)
+ }
+ }
+
+ private def isPossibleMaster(remoteUrl: Address) = {
+ masterUrls.map(s => Master.toAkkaUrl(s))
+ .map(u => AddressFromURIString(u).hostPort)
+ .contains(remoteUrl.hostPort)
}
override def receive = {
@@ -135,22 +144,16 @@ private[spark] class Client(
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 =>
- logWarning("Connection to master failed; waiting for master to reconnect...")
- markDisconnected()
-
- case RemoteClientDisconnected(transport, address) if address == masterAddress =>
- logWarning("Connection to master failed; waiting for master to reconnect...")
+ case DisassociatedEvent(_, address, _) if address == masterAddress =>
+ logWarning(s"Connection to $address failed; waiting for master to reconnect...")
markDisconnected()
- case RemoteClientShutdown(transport, address) if address == masterAddress =>
- logWarning("Connection to master failed; waiting for master to reconnect...")
- markDisconnected()
+ case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) =>
+ logWarning(s"Could not connect to $address: $cause")
case StopClient =>
markDead()
@@ -184,7 +187,7 @@ private[spark] class Client(
def stop() {
if (actor != null) {
try {
- val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+ val timeout = AkkaUtils.askTimeout(conf)
val future = actor.ask(StopClient)(timeout)
Await.result(future, timeout)
} catch {
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 5b62d3ba6c..28ebbdc66b 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
@@ -18,7 +18,7 @@
package org.apache.spark.deploy.client
import org.apache.spark.util.{Utils, AkkaUtils}
-import org.apache.spark.{Logging}
+import org.apache.spark.{SparkConf, SparkContext, Logging}
import org.apache.spark.deploy.{Command, ApplicationDescription}
private[spark] object TestClient {
@@ -45,11 +45,13 @@ private[spark] object TestClient {
def main(args: Array[String]) {
val url = args(0)
- val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
+ val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0,
+ conf = new SparkConf)
val desc = new ApplicationDescription(
- "TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home", "ignored")
+ "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()),
+ "dummy-spark-home", "ignored")
val listener = new TestListener
- val client = new Client(actorSystem, Array(url), desc, listener)
+ val client = new Client(actorSystem, Array(url), desc, listener, new SparkConf)
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 5150b7c7de..3e26379166 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
@@ -28,7 +28,8 @@ private[spark] class ApplicationInfo(
val desc: ApplicationDescription,
val submitDate: Date,
val driver: ActorRef,
- val appUiUrl: String)
+ val appUiUrl: String,
+ defaultCores: Int)
extends Serializable {
@transient var state: ApplicationState.Value = _
@@ -81,7 +82,9 @@ private[spark] class ApplicationInfo(
}
}
- def coresLeft: Int = desc.maxCores - coresGranted
+ private val myMaxCores = desc.maxCores.getOrElse(defaultCores)
+
+ def coresLeft: Int = myMaxCores - coresGranted
private var _retryCount = 0
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 fedf879eff..67e6c5d66a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationState.scala
@@ -17,8 +17,7 @@
package org.apache.spark.deploy.master
-private[spark] object ApplicationState
- extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED", "UNKNOWN") {
+private[spark] object ApplicationState extends Enumeration {
type ApplicationState = Value
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
index c0849ef324..043945a211 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala
@@ -65,7 +65,7 @@ private[spark] class FileSystemPersistenceEngine(
(apps, workers)
}
- private def serializeIntoFile(file: File, value: Serializable) {
+ private def serializeIntoFile(file: File, value: AnyRef) {
val created = file.createNewFile()
if (!created) { throw new IllegalStateException("Could not create file: " + file) }
@@ -77,13 +77,13 @@ private[spark] class FileSystemPersistenceEngine(
out.close()
}
- def deserializeFromFile[T <: Serializable](file: File)(implicit m: Manifest[T]): T = {
+ def deserializeFromFile[T](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 clazz = m.runtimeClass.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/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index cd916672ac..6617b7100f 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,21 +17,19 @@
package org.apache.spark.deploy.master
-import java.util.Date
import java.text.SimpleDateFormat
+import java.util.Date
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.concurrent.Await
+import scala.concurrent.duration._
import akka.actor._
-import akka.actor.Terminated
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
+import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
import akka.serialization.SerializationExtension
-import akka.util.duration._
-import akka.util.{Duration, Timeout}
-import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.{SparkConf, SparkContext, Logging, SparkException}
import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.MasterMessages._
@@ -40,12 +38,16 @@ import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.util.{AkkaUtils, Utils}
private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging {
+ import context.dispatcher // to use Akka's scheduler.schedule()
+
+ val conf = new SparkConf
+
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")
+ val WORKER_TIMEOUT = conf.get("spark.worker.timeout", "60").toLong * 1000
+ val RETAINED_APPLICATIONS = conf.get("spark.deploy.retainedApplications", "200").toInt
+ val REAPER_ITERATIONS = conf.get("spark.dead.worker.persistence", "15").toInt
+ val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "")
+ val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE")
var nextAppNumber = 0
val workers = new HashSet[WorkerInfo]
@@ -61,12 +63,10 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
val waitingApps = new ArrayBuffer[ApplicationInfo]
val completedApps = new ArrayBuffer[ApplicationInfo]
- var firstApp: Option[ApplicationInfo] = None
-
Utils.checkHost(host, "Expected hostname")
- val masterMetricsSystem = MetricsSystem.createMetricsSystem("master")
- val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications")
+ val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf)
+ val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf)
val masterSource = new MasterSource(this)
val webUi = new MasterWebUI(this, webUiPort)
@@ -88,12 +88,18 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
// 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
+ val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true)
+
+ // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue)
+ val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue)
+ if (defaultCores < 1) {
+ throw new SparkException("spark.deploy.defaultCores must be positive")
+ }
override def preStart() {
logInfo("Starting Spark master at " + masterUrl)
// Listen for remote client disconnection events, since they don't go through Akka's watch()
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
webUi.start()
masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get
context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut)
@@ -105,7 +111,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
persistenceEngine = RECOVERY_MODE match {
case "ZOOKEEPER" =>
logInfo("Persisting recovery state to ZooKeeper")
- new ZooKeeperPersistenceEngine(SerializationExtension(context.system))
+ new ZooKeeperPersistenceEngine(SerializationExtension(context.system), conf)
case "FILESYSTEM" =>
logInfo("Persisting recovery state to directory: " + RECOVERY_DIR)
new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system))
@@ -113,13 +119,12 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
new BlackHolePersistenceEngine()
}
- leaderElectionAgent = context.actorOf(Props(
- RECOVERY_MODE match {
+ leaderElectionAgent = RECOVERY_MODE match {
case "ZOOKEEPER" =>
- new ZooKeeperLeaderElectionAgent(self, masterUrl)
+ context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl, conf))
case _ =>
- new MonarchyLeaderAgent(self)
- }))
+ context.actorOf(Props(classOf[MonarchyLeaderAgent], self))
+ }
}
override def preRestart(reason: Throwable, message: Option[Any]) {
@@ -142,9 +147,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
RecoveryState.ALIVE
else
RecoveryState.RECOVERING
-
logInfo("I have been elected leader! New state: " + state)
-
if (state == RecoveryState.RECOVERING) {
beginRecovery(storedApps, storedWorkers)
context.system.scheduler.scheduleOnce(WORKER_TIMEOUT millis) { completeRecovery() }
@@ -156,7 +159,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
System.exit(0)
}
- case RegisterWorker(id, host, workerPort, cores, memory, webUiPort, publicAddress) => {
+ case RegisterWorker(id, workerHost, workerPort, cores, memory, workerWebUiPort, publicAddress) => {
logInfo("Registering worker %s:%d with %d cores, %s RAM".format(
host, workerPort, cores, Utils.megabytesToString(memory)))
if (state == RecoveryState.STANDBY) {
@@ -164,9 +167,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
} else if (idToWorker.contains(id)) {
sender ! RegisterWorkerFailed("Duplicate worker ID")
} else {
- val worker = new WorkerInfo(id, host, port, cores, memory, sender, webUiPort, publicAddress)
+ val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory,
+ sender, workerWebUiPort, publicAddress)
registerWorker(worker)
- context.watch(sender) // This doesn't work with remote actors but helps for testing
persistenceEngine.addWorker(worker)
sender ! RegisteredWorker(masterUrl, masterWebUiUrl)
schedule()
@@ -181,7 +184,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
val app = createApplication(description, sender)
registerApplication(app)
logInfo("Registered app " + description.name + " with ID " + app.id)
- context.watch(sender) // This doesn't work with remote actors but helps for testing
persistenceEngine.addApplication(app)
sender ! RegisteredApplication(app.id, masterUrl)
schedule()
@@ -257,23 +259,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
if (canCompleteRecovery) { completeRecovery() }
}
- case Terminated(actor) => {
- // The disconnected actor could've been either a worker or an app; remove whichever of
- // those we have an entry for in the corresponding actor hashmap
- 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) => {
+ case DisassociatedEvent(_, address, _) => {
// The disconnected client could've been either a worker or an app; remove whichever it was
+ logInfo(s"$address got disassociated, removing it.")
addressToWorker.get(address).foreach(removeWorker)
addressToApp.get(address).foreach(finishApplication)
if (state == RecoveryState.RECOVERING && canCompleteRecovery) { completeRecovery() }
@@ -444,7 +432,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
val now = System.currentTimeMillis()
val date = new Date(now)
- new ApplicationInfo(now, newApplicationId(date), desc, date, driver, desc.appUiUrl)
+ new ApplicationInfo(
+ now, newApplicationId(date), desc, date, driver, desc.appUiUrl, defaultCores)
}
def registerApplication(app: ApplicationInfo): Unit = {
@@ -459,14 +448,6 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
idToApp(app.id) = app
actorToApp(app.driver) = app
addressToApp(appAddress) = app
- if (firstApp == None) {
- firstApp = Some(app)
- }
- // TODO: What is firstApp?? Can we remove it?
- val workersAlive = workers.filter(_.state == WorkerState.ALIVE).toArray
- if (workersAlive.size > 0 && !workersAlive.exists(_.memoryFree >= app.desc.memoryPerSlave)) {
- logWarning("Could not find any workers with enough memory for " + firstApp.get.id)
- }
waitingApps += app
}
@@ -523,41 +504,42 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act
removeWorker(worker)
} else {
if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT))
- workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it
+ workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it
}
}
}
}
private[spark] object Master {
- private val systemName = "sparkMaster"
+ val systemName = "sparkMaster"
private val actorName = "Master"
- private val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
+ val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r
def main(argStrings: Array[String]) {
- val args = new MasterArguments(argStrings)
- val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort)
+ val conf = new SparkConf
+ val args = new MasterArguments(argStrings, conf)
+ val (actorSystem, _, _) = startSystemAndActor(args.host, args.port, args.webUiPort, conf)
actorSystem.awaitTermination()
}
- /** Returns an `akka://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */
+ /** Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */
def toAkkaUrl(sparkUrl: String): String = {
sparkUrl match {
case sparkUrlRegex(host, port) =>
- "akka://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
+ "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName)
case _ =>
throw new SparkException("Invalid master URL: " + sparkUrl)
}
}
- def startSystemAndActor(host: String, port: Int, webUiPort: Int): (ActorSystem, Int, Int) = {
- val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port)
- val actor = actorSystem.actorOf(Props(new Master(host, boundPort, webUiPort)), name = actorName)
- val 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]
+ def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf)
+ : (ActorSystem, Int, Int) =
+ {
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf)
+ val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName)
+ val timeout = AkkaUtils.askTimeout(conf)
+ val respFuture = actor.ask(RequestWebUIPort)(timeout)
+ val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse]
(actorSystem, boundPort, resp.webUIBoundPort)
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 9d89b455fb..e7f3224091 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -18,16 +18,17 @@
package org.apache.spark.deploy.master
import org.apache.spark.util.{Utils, IntParam}
+import org.apache.spark.SparkConf
/**
* Command-line parser for the master.
*/
-private[spark] class MasterArguments(args: Array[String]) {
+private[spark] class MasterArguments(args: Array[String], conf: SparkConf) {
var host = Utils.localHostName()
var port = 7077
var webUiPort = 8080
-
- // Check for settings in environment variables
+
+ // Check for settings in environment variables
if (System.getenv("SPARK_MASTER_HOST") != null) {
host = System.getenv("SPARK_MASTER_HOST")
}
@@ -37,8 +38,8 @@ private[spark] class MasterArguments(args: Array[String]) {
if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) {
webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt
}
- if (System.getProperty("master.ui.port") != null) {
- webUiPort = System.getProperty("master.ui.port").toInt
+ if (conf.contains("master.ui.port")) {
+ webUiPort = conf.get("master.ui.port").toInt
}
parse(args.toList)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
index b91be821f0..256a5a7c28 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryState.scala
@@ -17,9 +17,7 @@
package org.apache.spark.deploy.master
-private[spark] object RecoveryState
- extends Enumeration("STANDBY", "ALIVE", "RECOVERING", "COMPLETING_RECOVERY") {
-
+private[spark] object RecoveryState extends Enumeration {
type MasterState = Value
val STANDBY, ALIVE, RECOVERING, COMPLETING_RECOVERY = Value
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
index 81e15c534f..999090ad74 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala
@@ -18,12 +18,12 @@
package org.apache.spark.deploy.master
import scala.collection.JavaConversions._
-import scala.concurrent.ops._
-import org.apache.spark.Logging
import org.apache.zookeeper._
-import org.apache.zookeeper.data.Stat
import org.apache.zookeeper.Watcher.Event.KeeperState
+import org.apache.zookeeper.data.Stat
+
+import org.apache.spark.{SparkConf, Logging}
/**
* Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry
@@ -33,10 +33,11 @@ import org.apache.zookeeper.Watcher.Event.KeeperState
* informed via zkDown().
*
* Additionally, all commands sent to ZooKeeper will be retried until they either fail too many
- * times or a semantic exception is thrown (e.g.., "node already exists").
+ * times or a semantic exception is thrown (e.g., "node already exists").
*/
-private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) extends Logging {
- val ZK_URL = System.getProperty("spark.deploy.zookeeper.url", "")
+private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher,
+ conf: SparkConf) extends Logging {
+ val ZK_URL = conf.get("spark.deploy.zookeeper.url", "")
val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE
val ZK_TIMEOUT_MILLIS = 30000
@@ -103,6 +104,7 @@ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) ext
connectToZooKeeper()
case KeeperState.Disconnected =>
logWarning("ZooKeeper disconnected, will retry...")
+ case s => // Do nothing
}
}
}
@@ -179,7 +181,7 @@ private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher) ext
} catch {
case e: KeeperException.NoNodeException => throw e
case e: KeeperException.NodeExistsException => throw e
- case e if n > 0 =>
+ case e: Exception if n > 0 =>
logError("ZooKeeper exception, " + n + " more retries...", e)
Thread.sleep(RETRY_WAIT_MILLIS)
retry(fn, n-1)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerState.scala
index c8d34f25e2..0b36ef6005 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,9 +17,7 @@
package org.apache.spark.deploy.master
-private[spark] object WorkerState
- extends Enumeration("ALIVE", "DEAD", "DECOMMISSIONED", "UNKNOWN") {
-
+private[spark] object WorkerState extends Enumeration {
type WorkerState = 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
index 7809013e83..77c23fb9fb 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala
@@ -21,16 +21,17 @@ import akka.actor.ActorRef
import org.apache.zookeeper._
import org.apache.zookeeper.Watcher.Event.EventType
+import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.deploy.master.MasterMessages._
-import org.apache.spark.Logging
-private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String)
+private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef,
+ masterUrl: String, conf: SparkConf)
extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging {
- val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
+ val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election"
private val watcher = new ZooKeeperWatcher()
- private val zk = new SparkZooKeeperSession(this)
+ private val zk = new SparkZooKeeperSession(this, conf)
private var status = LeadershipStatus.NOT_LEADER
private var myLeaderFile: String = _
private var leaderUrl: String = _
@@ -105,7 +106,7 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, mas
// We found a different master file pointing to this process.
// This can happen in the following two cases:
// (1) The master process was restarted on the same node.
- // (2) The ZK server died between creating the node and returning the name of the node.
+ // (2) The ZK server died between creating the file and returning the name of the file.
// For this case, we will end up creating a second file, and MUST explicitly delete the
// first one, since our ZK session is still open.
// Note that this deletion will cause a NodeDeleted event to be fired so we check again for
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index a0233a7271..52000d4f9c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -17,19 +17,19 @@
package org.apache.spark.deploy.master
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
import org.apache.zookeeper._
import akka.serialization.Serialization
-class ZooKeeperPersistenceEngine(serialization: Serialization)
+class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
extends PersistenceEngine
with SparkZooKeeperWatcher
with Logging
{
- val WORKING_DIR = System.getProperty("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
+ val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
- val zk = new SparkZooKeeperSession(this)
+ val zk = new SparkZooKeeperSession(this, conf)
zk.connect()
@@ -70,15 +70,15 @@ class ZooKeeperPersistenceEngine(serialization: Serialization)
(apps, workers)
}
- private def serializeIntoFile(path: String, value: Serializable) {
+ private def serializeIntoFile(path: String, value: AnyRef) {
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 = {
+ def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): T = {
val fileData = zk.getData("/spark/master_status/" + filename)
- val clazz = m.erasure.asInstanceOf[Class[T]]
+ val clazz = m.runtimeClass.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/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index f4e574d15d..dbb0cb90f5 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
@@ -17,31 +17,28 @@
package org.apache.spark.deploy.master.ui
+import scala.concurrent.Await
import scala.xml.Node
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.util.duration._
-
import javax.servlet.http.HttpServletRequest
-
import net.liftweb.json.JsonAST.JValue
-import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
import org.apache.spark.deploy.JsonProtocol
+import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
import org.apache.spark.deploy.master.ExecutorInfo
import org.apache.spark.ui.UIUtils
import org.apache.spark.util.Utils
private[spark] class ApplicationPage(parent: MasterWebUI) {
val master = parent.masterActorRef
- implicit val timeout = parent.timeout
+ val timeout = parent.timeout
/** Executor details for a particular application */
def renderJson(request: HttpServletRequest): JValue = {
val appId = request.getParameter("appId")
val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
- val state = Await.result(stateFuture, 30 seconds)
+ val state = Await.result(stateFuture, timeout)
val app = state.activeApps.find(_.id == appId).getOrElse({
state.completedApps.find(_.id == appId).getOrElse(null)
})
@@ -52,7 +49,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
def render(request: HttpServletRequest): Seq[Node] = {
val appId = request.getParameter("appId")
val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
- val state = Await.result(stateFuture, 30 seconds)
+ val state = Await.result(stateFuture, timeout)
val app = state.activeApps.find(_.id == appId).getOrElse({
state.completedApps.find(_.id == appId).getOrElse(null)
})
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
index d7a57229b0..4ef762892c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
@@ -17,37 +17,33 @@
package org.apache.spark.deploy.master.ui
-import javax.servlet.http.HttpServletRequest
-
+import scala.concurrent.Await
import scala.xml.Node
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.util.duration._
-
+import javax.servlet.http.HttpServletRequest
import net.liftweb.json.JsonAST.JValue
-import org.apache.spark.deploy.DeployWebUI
+import org.apache.spark.deploy.{DeployWebUI, JsonProtocol}
import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState}
-import org.apache.spark.deploy.JsonProtocol
import org.apache.spark.deploy.master.{ApplicationInfo, WorkerInfo}
import org.apache.spark.ui.UIUtils
import org.apache.spark.util.Utils
private[spark] class IndexPage(parent: MasterWebUI) {
val master = parent.masterActorRef
- implicit val timeout = parent.timeout
+ val timeout = parent.timeout
def renderJson(request: HttpServletRequest): JValue = {
val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
- val state = Await.result(stateFuture, 30 seconds)
+ val state = Await.result(stateFuture, timeout)
JsonProtocol.writeMasterState(state)
}
/** Index view listing applications and executors */
def render(request: HttpServletRequest): Seq[Node] = {
val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
- val state = Await.result(stateFuture, 30 seconds)
+ val state = Await.result(stateFuture, timeout)
val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory")
val workers = state.workers.sortBy(_.id)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
index f4df729e87..ead35662fc 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala
@@ -17,25 +17,21 @@
package org.apache.spark.deploy.master.ui
-import akka.util.Duration
-
import javax.servlet.http.HttpServletRequest
-
import org.eclipse.jetty.server.{Handler, Server}
-import org.apache.spark.{Logging}
+import org.apache.spark.Logging
import org.apache.spark.deploy.master.Master
import org.apache.spark.ui.JettyUtils
import org.apache.spark.ui.JettyUtils._
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{AkkaUtils, Utils}
/**
* Web UI server for the standalone master.
*/
private[spark]
class MasterWebUI(val master: Master, requestedPort: Int) extends Logging {
- implicit val timeout = Duration.create(
- System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+ val timeout = AkkaUtils.askTimeout(master.conf)
val host = Utils.localHostName()
val port = requestedPort
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 216d9d44ac..fcaf4e92b1 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -17,23 +17,22 @@
package org.apache.spark.deploy.worker
+import java.io.File
import java.text.SimpleDateFormat
import java.util.Date
-import java.io.File
import scala.collection.mutable.HashMap
+import scala.concurrent.duration._
import akka.actor._
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
-import akka.util.duration._
-
-import org.apache.spark.Logging
+import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
+import org.apache.spark.{Logging, SparkConf, SparkException}
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}
+import org.apache.spark.util.{AkkaUtils, Utils}
/**
* @param masterUrls Each url should look like spark://host:port.
@@ -45,8 +44,10 @@ private[spark] class Worker(
cores: Int,
memory: Int,
masterUrls: Array[String],
- workDirPath: String = null)
+ workDirPath: String = null,
+ val conf: SparkConf)
extends Actor with Logging {
+ import context.dispatcher
Utils.checkHost(host, "Expected hostname")
assert (port > 0)
@@ -54,7 +55,7 @@ private[spark] class Worker(
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs
// Send a heartbeat every (heartbeat timeout) / 4 milliseconds
- val HEARTBEAT_MILLIS = System.getProperty("spark.worker.timeout", "60").toLong * 1000 / 4
+ val HEARTBEAT_MILLIS = conf.get("spark.worker.timeout", "60").toLong * 1000 / 4
val REGISTRATION_TIMEOUT = 20.seconds
val REGISTRATION_RETRIES = 3
@@ -63,7 +64,8 @@ private[spark] class Worker(
var masterIndex = 0
val masterLock: Object = new Object()
- var master: ActorRef = null
+ var master: ActorSelection = null
+ var masterAddress: Address = null
var activeMasterUrl: String = ""
var activeMasterWebUiUrl : String = ""
@volatile var registered = false
@@ -82,7 +84,7 @@ private[spark] class Worker(
var coresUsed = 0
var memoryUsed = 0
- val metricsSystem = MetricsSystem.createMetricsSystem("worker")
+ val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf)
val workerSource = new WorkerSource(this)
def coresFree: Int = cores - coresUsed
@@ -114,7 +116,7 @@ private[spark] class Worker(
logInfo("Spark home: " + sparkHome)
createWorkDir()
webUi = new WorkerWebUI(this, workDir, Some(webUiPort))
-
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
webUi.start()
registerWithMaster()
@@ -126,9 +128,13 @@ private[spark] class Worker(
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
+ master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl))
+ masterAddress = activeMasterUrl match {
+ case Master.sparkUrlRegex(_host, _port) =>
+ Address("akka.tcp", Master.systemName, _host, _port.toInt)
+ case x =>
+ throw new SparkException("Invalid spark URL: " + x)
+ }
connected = true
}
}
@@ -136,7 +142,7 @@ private[spark] class Worker(
def tryRegisterAllMasters() {
for (masterUrl <- masterUrls) {
logInfo("Connecting to master " + masterUrl + "...")
- val actor = context.actorFor(Master.toAkkaUrl(masterUrl))
+ val actor = context.actorSelection(Master.toAkkaUrl(masterUrl))
actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get,
publicAddress)
}
@@ -175,7 +181,6 @@ private[spark] class Worker(
case MasterChanged(masterUrl, masterWebUiUrl) =>
logInfo("Master has changed, new master is at " + masterUrl)
- context.unwatch(master)
changeMaster(masterUrl, masterWebUiUrl)
val execs = executors.values.
@@ -234,13 +239,8 @@ private[spark] class Worker(
}
}
- 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 =>
+ case x: DisassociatedEvent if x.remoteAddress == masterAddress =>
+ logInfo(s"$x Disassociated !")
masterDisconnected()
case RequestWorkerState => {
@@ -267,6 +267,7 @@ private[spark] class Worker(
}
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,
@@ -275,13 +276,16 @@ private[spark] object Worker {
}
def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int,
- masterUrls: Array[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 conf = new SparkConf
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,
- masterUrls, workDir)), name = "Worker")
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port,
+ conf = conf)
+ actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory,
+ masterUrls, workDir, conf), name = "Worker")
(actorSystem, boundPort)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
index d2d3617498..0d59048313 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala
@@ -21,9 +21,10 @@ import javax.servlet.http.HttpServletRequest
import scala.xml.Node
-import akka.dispatch.Await
+import scala.concurrent.duration._
+import scala.concurrent.Await
+
import akka.pattern.ask
-import akka.util.duration._
import net.liftweb.json.JsonAST.JValue
@@ -41,13 +42,13 @@ private[spark] class IndexPage(parent: WorkerWebUI) {
def renderJson(request: HttpServletRequest): JValue = {
val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse]
- val workerState = Await.result(stateFuture, 30 seconds)
+ val workerState = Await.result(stateFuture, timeout)
JsonProtocol.writeWorkerState(workerState)
}
def render(request: HttpServletRequest): Seq[Node] = {
val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse]
- val workerState = Await.result(stateFuture, 30 seconds)
+ val workerState = Await.result(stateFuture, timeout)
val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs")
val runningExecutorTable =
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 800f1cafcc..c382034c99 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -17,20 +17,16 @@
package org.apache.spark.deploy.worker.ui
-import akka.util.{Duration, Timeout}
-
-import java.io.{FileInputStream, File}
+import java.io.File
import javax.servlet.http.HttpServletRequest
-
import org.eclipse.jetty.server.{Handler, Server}
+import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.deploy.worker.Worker
-import org.apache.spark.{Logging}
-import org.apache.spark.ui.JettyUtils
+import org.apache.spark.ui.{JettyUtils, UIUtils}
import org.apache.spark.ui.JettyUtils._
-import org.apache.spark.ui.UIUtils
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{AkkaUtils, Utils}
/**
* Web UI server for the standalone worker.
@@ -38,11 +34,10 @@ import org.apache.spark.util.Utils
private[spark]
class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None)
extends Logging {
- implicit val timeout = Timeout(
- Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds"))
+ val timeout = AkkaUtils.askTimeout(worker.conf)
val host = Utils.localHostName()
val port = requestedPort.getOrElse(
- System.getProperty("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt)
+ worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt)
var server: Option[Server] = None
var boundPort: Option[Int] = None
@@ -145,12 +140,12 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I
<body>
{linkToMaster}
<div>
- <div style="float:left;width:40%">{backButton}</div>
+ <div style="float:left; margin-right:10px">{backButton}</div>
<div style="float:left;">{range}</div>
- <div style="float:right;">{nextButton}</div>
+ <div style="float:right; margin-left:10px">{nextButton}</div>
</div>
<br />
- <div style="height:500px;overflow:auto;padding:5px;">
+ <div style="height:500px; overflow:auto; padding:5px;">
<pre>{logText}</pre>
</div>
</body>
diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
index 8332631838..53a2b94a52 100644
--- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
+++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
@@ -19,15 +19,14 @@ package org.apache.spark.executor
import java.nio.ByteBuffer
-import akka.actor.{ActorRef, Actor, Props, Terminated}
-import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientShutdown, RemoteClientDisconnected}
+import akka.actor._
+import akka.remote._
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, SparkContext, Logging}
import org.apache.spark.TaskState.TaskState
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
import org.apache.spark.util.{Utils, AkkaUtils}
-
private[spark] class CoarseGrainedExecutorBackend(
driverUrl: String,
executorId: String,
@@ -40,14 +39,13 @@ private[spark] class CoarseGrainedExecutorBackend(
Utils.checkHostPort(hostPort, "Expected hostport")
var executor: Executor = null
- var driver: ActorRef = null
+ var driver: ActorSelection = null
override def preStart() {
logInfo("Connecting to driver: " + driverUrl)
- driver = context.actorFor(driverUrl)
+ driver = context.actorSelection(driverUrl)
driver ! RegisterExecutor(executorId, hostPort, cores)
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
- context.watch(driver) // Doesn't work with remote actors, but useful for testing
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
}
override def receive = {
@@ -77,8 +75,8 @@ private[spark] class CoarseGrainedExecutorBackend(
executor.killTask(taskId)
}
- case Terminated(_) | RemoteClientDisconnected(_, _) | RemoteClientShutdown(_, _) =>
- logError("Driver terminated or disconnected! Shutting down.")
+ case x: DisassociatedEvent =>
+ logError(s"Driver $x disassociated! Shutting down.")
System.exit(1)
case StopExecutor =>
@@ -99,12 +97,13 @@ private[spark] object CoarseGrainedExecutorBackend {
// Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor
// before getting started with all our system properties, etc
- val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0)
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0,
+ indestructible = true, conf = new SparkConf)
// set it
val sparkHostPort = hostname + ":" + boundPort
- System.setProperty("spark.hostPort", sparkHostPort)
- val actor = actorSystem.actorOf(
- Props(new CoarseGrainedExecutorBackend(driverUrl, executorId, sparkHostPort, cores)),
+// conf.set("spark.hostPort", sparkHostPort)
+ actorSystem.actorOf(
+ Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores),
name = "Executor")
actorSystem.awaitTermination()
}
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 5c9bb9db1c..e51d274d33 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -48,8 +48,6 @@ private[spark] class Executor(
private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0))
- initLogging()
-
// No ip or host:port - just hostname
Utils.checkHost(slaveHostname, "Expected executed slave to be a hostname")
// must not have port specified.
@@ -58,16 +56,17 @@ private[spark] class Executor(
// Make sure the local hostname we report matches the cluster scheduler's name for this host
Utils.setCustomHostname(slaveHostname)
- // Set spark.* system properties from executor arg
- for ((key, value) <- properties) {
- System.setProperty(key, value)
- }
+ // Set spark.* properties from executor arg
+ val conf = new SparkConf(false)
+ conf.setAll(properties)
// If we are in yarn mode, systems can have different disk layouts so we must set it
// to what Yarn on this system said was available. This will be used later when SparkEnv
// created.
- if (java.lang.Boolean.valueOf(System.getenv("SPARK_YARN_MODE"))) {
- System.setProperty("spark.local.dir", getYarnLocalDirs())
+ if (java.lang.Boolean.valueOf(
+ System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))))
+ {
+ conf.set("spark.local.dir", getYarnLocalDirs())
}
// Create our ClassLoader and set it on this thread
@@ -108,7 +107,7 @@ private[spark] class Executor(
// Initialize Spark environment (using system properties read above)
private val env = {
if (!isLocal) {
- val _env = SparkEnv.createFromSystemProperties(executorId, slaveHostname, 0,
+ val _env = SparkEnv.create(conf, executorId, slaveHostname, 0,
isDriver = false, isLocal = false)
SparkEnv.set(_env)
_env.metricsSystem.registerSource(executorSource)
@@ -121,7 +120,7 @@ private[spark] class Executor(
// 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")
+ env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size")
}
// Start worker thread pool
@@ -142,11 +141,6 @@ private[spark] class Executor(
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)
}
}
@@ -168,6 +162,8 @@ private[spark] class Executor(
class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer)
extends Runnable {
+ object TaskKilledException extends Exception
+
@volatile private var killed = false
@volatile private var task: Task[Any] = _
@@ -201,9 +197,11 @@ private[spark] class Executor(
// 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
+ // Throw an exception rather than returning, because returning within a try{} block
+ // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl
+ // exception will be caught by the catch block, leading to an incorrect ExceptionFailure
+ // for the task.
+ throw TaskKilledException
}
attemptedTask = Some(task)
@@ -217,23 +215,25 @@ private[spark] class Executor(
// 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
+ throw TaskKilledException
}
+ val resultSer = SparkEnv.get.serializer.newInstance()
+ val beforeSerialization = System.currentTimeMillis()
+ val valueBytes = resultSer.serialize(value)
+ val afterSerialization = System.currentTimeMillis()
+
for (m <- task.metrics) {
m.hostname = Utils.localHostName()
m.executorDeserializeTime = (taskStart - startTime).toInt
m.executorRunTime = (taskFinish - taskStart).toInt
m.jvmGCTime = gcTime - startGCTime
+ m.resultSerializationTime = (afterSerialization - beforeSerialization).toInt
}
- // TODO I'd also like to track the time it takes to serialize the task results, but that is
- // huge headache, b/c we need to serialize the task metrics first. If TaskMetrics had a
- // custom serialized format, we could just change the relevants bytes in the byte buffer
+
val accumUpdates = Accumulators.values
- val directResult = new DirectTaskResult(value, accumUpdates, task.metrics.getOrElse(null))
+ val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.getOrElse(null))
val serializedDirectResult = ser.serialize(directResult)
logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit)
val serializedResult = {
@@ -257,6 +257,11 @@ private[spark] class Executor(
execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason))
}
+ case TaskKilledException => {
+ logInfo("Executor killed task " + taskId)
+ execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled))
+ }
+
case t: Throwable => {
val serviceTime = (System.currentTimeMillis() - taskStart).toInt
val metrics = attemptedTask.flatMap(t => t.metrics)
@@ -299,7 +304,7 @@ private[spark] class Executor(
* new classes defined by the REPL as the user types code
*/
private def addReplClassLoaderIfNeeded(parent: ClassLoader): ClassLoader = {
- val classUri = System.getProperty("spark.repl.class.uri")
+ val classUri = conf.get("spark.repl.class.uri", null)
if (classUri != null) {
logInfo("Using REPL class URI: " + classUri)
try {
@@ -327,12 +332,12 @@ private[spark] class Executor(
// 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))
+ Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf)
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))
+ Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf)
currentJars(name) = timestamp
// Add it to our class loader
val localName = name.split("/").last
diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
index 0b4892f98f..bb1471d9ee 100644
--- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
+++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
@@ -44,6 +44,11 @@ class TaskMetrics extends Serializable {
var jvmGCTime: Long = _
/**
+ * Amount of time spent serializing the task result
+ */
+ var resultSerializationTime: Long = _
+
+ /**
* If this task reads from shuffle output, metrics on getting shuffle data will be collected here
*/
var shuffleReadMetrics: Option[ShuffleReadMetrics] = None
@@ -61,50 +66,53 @@ object TaskMetrics {
class ShuffleReadMetrics extends Serializable {
/**
- * Time when shuffle finishs
+ * Absolute time when this task finished reading shuffle data
*/
var shuffleFinishTime: Long = _
/**
- * Total number of blocks fetched in a shuffle (remote or local)
+ * Number of blocks fetched in this shuffle by this task (remote or local)
*/
var totalBlocksFetched: Int = _
/**
- * Number of remote blocks fetched in a shuffle
+ * Number of remote blocks fetched in this shuffle by this task
*/
var remoteBlocksFetched: Int = _
/**
- * Local blocks fetched in a shuffle
+ * Number of local blocks fetched in this shuffle by this task
*/
var localBlocksFetched: Int = _
/**
- * Total time that is spent blocked waiting for shuffle to fetch data
+ * Time the task spent waiting for remote shuffle blocks. This only includes the time
+ * blocking on shuffle input data. For instance if block B is being fetched while the task is
+ * still not finished processing block A, it is not considered to be blocking on block B.
*/
var fetchWaitTime: Long = _
/**
- * The total amount of time for all the shuffle fetches. This adds up time from overlapping
- * shuffles, so can be longer than task time
+ * Total time spent fetching remote shuffle blocks. This aggregates the time spent fetching all
+ * input blocks. Since block fetches are both pipelined and parallelized, this can
+ * exceed fetchWaitTime and executorRunTime.
*/
var remoteFetchTime: Long = _
/**
- * Total number of remote bytes read from a shuffle
+ * Total number of remote bytes read from the shuffle by this task
*/
var remoteBytesRead: Long = _
}
class ShuffleWriteMetrics extends Serializable {
/**
- * Number of bytes written for a shuffle
+ * Number of bytes written for the shuffle by this task
*/
var shuffleBytesWritten: Long = _
/**
- * Time spent blocking on writes to disk or buffer cache, in nanoseconds.
+ * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds
*/
var shuffleWriteTime: Long = _
}
diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 570a979b56..a1e98845f6 100644
--- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
+++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
@@ -22,6 +22,7 @@ import java.io.{InputStream, OutputStream}
import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream}
+import org.apache.spark.{SparkEnv, SparkConf}
/**
@@ -37,15 +38,15 @@ trait CompressionCodec {
private[spark] object CompressionCodec {
-
- def createCodec(): CompressionCodec = {
- createCodec(System.getProperty(
+ def createCodec(conf: SparkConf): CompressionCodec = {
+ createCodec(conf, conf.get(
"spark.io.compression.codec", classOf[LZFCompressionCodec].getName))
}
- def createCodec(codecName: String): CompressionCodec = {
- Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
- .newInstance().asInstanceOf[CompressionCodec]
+ def createCodec(conf: SparkConf, codecName: String): CompressionCodec = {
+ val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader)
+ .getConstructor(classOf[SparkConf])
+ ctor.newInstance(conf).asInstanceOf[CompressionCodec]
}
}
@@ -53,7 +54,7 @@ private[spark] object CompressionCodec {
/**
* LZF implementation of [[org.apache.spark.io.CompressionCodec]].
*/
-class LZFCompressionCodec extends CompressionCodec {
+class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec {
override def compressedOutputStream(s: OutputStream): OutputStream = {
new LZFOutputStream(s).setFinishBlockOnFlush(true)
@@ -67,10 +68,10 @@ class LZFCompressionCodec extends CompressionCodec {
* Snappy implementation of [[org.apache.spark.io.CompressionCodec]].
* Block size can be configured by spark.io.compression.snappy.block.size.
*/
-class SnappyCompressionCodec extends CompressionCodec {
+class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec {
override def compressedOutputStream(s: OutputStream): OutputStream = {
- val blockSize = System.getProperty("spark.io.compression.snappy.block.size", "32768").toInt
+ val blockSize = conf.get("spark.io.compression.snappy.block.size", "32768").toInt
new SnappyOutputStream(s, blockSize)
}
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 caab748d60..6f9f29969e 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala
@@ -26,7 +26,6 @@ import scala.util.matching.Regex
import org.apache.spark.Logging
private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging {
- initLogging()
val DEFAULT_PREFIX = "*"
val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r
diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
index bec0c83be8..9930537b34 100644
--- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
@@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit
import scala.collection.mutable
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.metrics.sink.{MetricsServlet, Sink}
import org.apache.spark.metrics.source.Source
@@ -62,10 +62,10 @@ import org.apache.spark.metrics.source.Source
*
* [options] is the specific property of this source or sink.
*/
-private[spark] class MetricsSystem private (val instance: String) extends Logging {
- initLogging()
+private[spark] class MetricsSystem private (val instance: String,
+ conf: SparkConf) extends Logging {
- val confFile = System.getProperty("spark.metrics.conf")
+ val confFile = conf.get("spark.metrics.conf", null)
val metricsConfig = new MetricsConfig(Option(confFile))
val sinks = new mutable.ArrayBuffer[Sink]
@@ -159,5 +159,6 @@ private[spark] object MetricsSystem {
}
}
- def createMetricsSystem(instance: String): MetricsSystem = new MetricsSystem(instance)
+ def createMetricsSystem(instance: String, conf: SparkConf): MetricsSystem =
+ new MetricsSystem(instance, conf)
}
diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
index 9c2fee4023..46c40d0a2a 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala
@@ -31,13 +31,13 @@ import scala.collection.mutable.SynchronizedMap
import scala.collection.mutable.SynchronizedQueue
import scala.collection.mutable.ArrayBuffer
-import akka.dispatch.{Await, Promise, ExecutionContext, Future}
-import akka.util.Duration
-import akka.util.duration._
-import org.apache.spark.util.Utils
+import scala.concurrent.{Await, Promise, ExecutionContext, Future}
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration._
+import org.apache.spark.util.Utils
-private[spark] class ConnectionManager(port: Int) extends Logging {
+private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Logging {
class MessageStatus(
val message: Message,
@@ -54,22 +54,22 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
private val selector = SelectorProvider.provider.openSelector()
private val handleMessageExecutor = new ThreadPoolExecutor(
- System.getProperty("spark.core.connection.handler.threads.min","20").toInt,
- System.getProperty("spark.core.connection.handler.threads.max","60").toInt,
- System.getProperty("spark.core.connection.handler.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+ conf.get("spark.core.connection.handler.threads.min", "20").toInt,
+ conf.get("spark.core.connection.handler.threads.max", "60").toInt,
+ conf.get("spark.core.connection.handler.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable]())
private val handleReadWriteExecutor = new ThreadPoolExecutor(
- System.getProperty("spark.core.connection.io.threads.min","4").toInt,
- System.getProperty("spark.core.connection.io.threads.max","32").toInt,
- System.getProperty("spark.core.connection.io.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+ conf.get("spark.core.connection.io.threads.min", "4").toInt,
+ conf.get("spark.core.connection.io.threads.max", "32").toInt,
+ conf.get("spark.core.connection.io.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable]())
// Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : which should be executed asap
private val handleConnectExecutor = new ThreadPoolExecutor(
- System.getProperty("spark.core.connection.connect.threads.min","1").toInt,
- System.getProperty("spark.core.connection.connect.threads.max","8").toInt,
- System.getProperty("spark.core.connection.connect.threads.keepalive","60").toInt, TimeUnit.SECONDS,
+ conf.get("spark.core.connection.connect.threads.min", "1").toInt,
+ conf.get("spark.core.connection.connect.threads.max", "8").toInt,
+ conf.get("spark.core.connection.connect.threads.keepalive", "60").toInt, TimeUnit.SECONDS,
new LinkedBlockingDeque[Runnable]())
private val serverChannel = ServerSocketChannel.open()
@@ -594,7 +594,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging {
private[spark] object ConnectionManager {
def main(args: Array[String]) {
- val manager = new ConnectionManager(9999)
+ val manager = new ConnectionManager(9999, new SparkConf)
manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
println("Received [" + msg + "] from [" + id + "]")
None
diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
index 8d9ad9604d..4f5742d29b 100644
--- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala
@@ -25,8 +25,8 @@ import scala.io.Source
import java.nio.ByteBuffer
import java.net.InetAddress
-import akka.dispatch.Await
-import akka.util.duration._
+import scala.concurrent.Await
+import scala.concurrent.duration._
private[spark] object ConnectionManagerTest extends Logging{
def main(args: Array[String]) {
diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
index 781715108b..1c9d6030d6 100644
--- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala
@@ -19,19 +19,19 @@ package org.apache.spark.network
import java.nio.ByteBuffer
import java.net.InetAddress
+import org.apache.spark.SparkConf
private[spark] object ReceiverTest {
-
def main(args: Array[String]) {
- val manager = new ConnectionManager(9999)
+ val manager = new ConnectionManager(9999, new SparkConf)
println("Started connection manager with id = " + manager.id)
-
- manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
+
+ manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
/*println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/
- val buffer = ByteBuffer.wrap("response".getBytes())
+ val buffer = ByteBuffer.wrap("response".getBytes)
Some(Message.createBufferMessage(buffer, msg.id))
})
- Thread.currentThread.join()
+ Thread.currentThread.join()
}
}
diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala
index 777574980f..dcbd183c88 100644
--- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala
+++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala
@@ -19,29 +19,29 @@ package org.apache.spark.network
import java.nio.ByteBuffer
import java.net.InetAddress
+import org.apache.spark.SparkConf
private[spark] object SenderTest {
-
def main(args: Array[String]) {
-
+
if (args.length < 2) {
println("Usage: SenderTest <target host> <target port>")
System.exit(1)
}
-
+
val targetHost = args(0)
val targetPort = args(1).toInt
val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort)
- val manager = new ConnectionManager(0)
+ val manager = new ConnectionManager(0, new SparkConf)
println("Started connection manager with id = " + manager.id)
- manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
+ manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => {
println("Received [" + msg + "] from [" + id + "]")
None
})
-
- val size = 100 * 1024 * 1024
+
+ val size = 100 * 1024 * 1024
val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte))
buffer.flip
@@ -50,7 +50,7 @@ private[spark] object SenderTest {
val count = 100
(0 until count).foreach(i => {
val dataMessage = Message.createBufferMessage(buffer.duplicate)
- val startTime = System.currentTimeMillis
+ val startTime = System.currentTimeMillis
/*println("Started timer at " + startTime)*/
val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) match {
case Some(response) =>
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 b1e1576dad..b729eb11c5 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
@@ -23,20 +23,20 @@ import io.netty.buffer.ByteBuf
import io.netty.channel.ChannelHandlerContext
import io.netty.util.CharsetUtil
-import org.apache.spark.Logging
+import org.apache.spark.{SparkContext, SparkConf, Logging}
import org.apache.spark.network.ConnectionManagerId
import scala.collection.JavaConverters._
import org.apache.spark.storage.BlockId
-private[spark] class ShuffleCopier extends Logging {
+private[spark] class ShuffleCopier(conf: SparkConf) extends Logging {
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
+ val connectTimeout = conf.get("spark.shuffle.netty.connect.timeout", "60000").toInt
val fc = new FileClient(handler, connectTimeout)
try {
@@ -104,10 +104,10 @@ private[spark] object ShuffleCopier extends Logging {
val threads = if (args.length > 3) args(3).toInt else 10
val copiers = Executors.newFixedThreadPool(80)
- val tasks = (for (i <- Range(0, threads)) yield {
+ val tasks = (for (i <- Range(0, threads)) yield {
Executors.callable(new Runnable() {
def run() {
- val copier = new ShuffleCopier()
+ val copier = new ShuffleCopier(new SparkConf)
copier.getBlock(host, port, blockId, echoResultCollectCallBack)
}
})
diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
index faaf837be0..d1c74a5063 100644
--- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala
@@ -21,6 +21,7 @@ import java.util.concurrent.atomic.AtomicLong
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.ExecutionContext.Implicits.global
+import scala.reflect.ClassTag
import org.apache.spark.{ComplexFutureAction, FutureAction, Logging}
@@ -28,7 +29,7 @@ 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 {
+class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging {
/**
* Returns a future for counting the number of elements in the RDD.
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 44ea573a7c..424354ae16 100644
--- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -17,6 +17,8 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext}
import org.apache.spark.storage.{BlockId, BlockManager}
@@ -25,7 +27,7 @@ private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends P
}
private[spark]
-class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[BlockId])
+class BlockRDD[T: ClassTag](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/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
index 0de22f0e06..87b950ba43 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala
@@ -18,6 +18,7 @@
package org.apache.spark.rdd
import java.io.{ObjectOutputStream, IOException}
+import scala.reflect.ClassTag
import org.apache.spark._
@@ -43,7 +44,7 @@ class CartesianPartition(
}
private[spark]
-class CartesianRDD[T: ClassManifest, U:ClassManifest](
+class CartesianRDD[T: ClassTag, U: ClassTag](
sc: SparkContext,
var rdd1 : RDD[T],
var rdd2 : RDD[U])
diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
index d3033ea4a6..6d4f46125f 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
@@ -17,15 +17,13 @@
package org.apache.spark.rdd
+import java.io.IOException
+import scala.reflect.ClassTag
import org.apache.spark._
+import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.hadoop.mapred.{FileInputFormat, SequenceFileInputFormat, JobConf, Reporter}
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.io.{NullWritable, BytesWritable}
-import org.apache.hadoop.util.ReflectionUtils
import org.apache.hadoop.fs.Path
-import java.io.{File, IOException, EOFException}
-import java.text.NumberFormat
private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
@@ -33,9 +31,11 @@ private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
* This RDD represents a RDD checkpoint file (similar to HadoopRDD).
*/
private[spark]
-class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: String)
+class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String)
extends RDD[T](sc, Nil) {
+ val broadcastedConf = sc.broadcast(new SerializableWritable(sc.hadoopConfiguration))
+
@transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration)
override def getPartitions: Array[Partition] = {
@@ -67,7 +67,7 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri
override def compute(split: Partition, context: TaskContext): Iterator[T] = {
val file = new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index))
- CheckpointRDD.readFromFile(file, context)
+ CheckpointRDD.readFromFile(file, broadcastedConf, context)
}
override def checkpoint() {
@@ -76,15 +76,18 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri
}
private[spark] object CheckpointRDD extends Logging {
-
def splitIdToFile(splitId: Int): String = {
"part-%05d".format(splitId)
}
- def writeToFile[T](path: String, blockSize: Int = -1)(ctx: TaskContext, iterator: Iterator[T]) {
+ def writeToFile[T](
+ path: String,
+ broadcastedConf: Broadcast[SerializableWritable[Configuration]],
+ blockSize: Int = -1
+ )(ctx: TaskContext, iterator: Iterator[T]) {
val env = SparkEnv.get
val outputDir = new Path(path)
- val fs = outputDir.getFileSystem(SparkHadoopUtil.get.newConfiguration())
+ val fs = outputDir.getFileSystem(broadcastedConf.value.value)
val finalOutputName = splitIdToFile(ctx.partitionId)
val finalOutputPath = new Path(outputDir, finalOutputName)
@@ -94,7 +97,7 @@ private[spark] object CheckpointRDD extends Logging {
throw new IOException("Checkpoint failed: temporary path " +
tempOutputPath + " already exists")
}
- val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+ val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
val fileOutputStream = if (blockSize < 0) {
fs.create(tempOutputPath, false, bufferSize)
@@ -121,10 +124,14 @@ private[spark] object CheckpointRDD extends Logging {
}
}
- def readFromFile[T](path: Path, context: TaskContext): Iterator[T] = {
+ def readFromFile[T](
+ path: Path,
+ broadcastedConf: Broadcast[SerializableWritable[Configuration]],
+ context: TaskContext
+ ): Iterator[T] = {
val env = SparkEnv.get
- val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration())
- val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
+ val fs = path.getFileSystem(broadcastedConf.value.value)
+ val bufferSize = env.conf.get("spark.buffer.size", "65536").toInt
val fileInputStream = fs.open(path, bufferSize)
val serializer = env.serializer.newInstance()
val deserializeStream = serializer.deserializeStream(fileInputStream)
@@ -146,8 +153,10 @@ private[spark] object CheckpointRDD extends Logging {
val sc = new SparkContext(cluster, "CheckpointRDD Test")
val rdd = sc.makeRDD(1 to 10, 10).flatMap(x => 1 to 10000)
val path = new Path(hdfsPath, "temp")
- val fs = path.getFileSystem(SparkHadoopUtil.get.newConfiguration())
- sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _)
+ val conf = SparkHadoopUtil.get.newConfiguration()
+ val fs = path.getFileSystem(conf)
+ val broadcastedConf = sc.broadcast(new SerializableWritable(conf))
+ sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf, 1024) _)
val cpRDD = new CheckpointRDD[Int](sc, path.toString)
assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same")
assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same")
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 911a002884..4ba4696fef 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -114,7 +114,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
map.changeValue(k, update)
}
- val ser = SparkEnv.get.serializerManager.get(serializerClass)
+ val ser = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)
for ((dep, depNum) <- split.deps.zipWithIndex) dep match {
case NarrowCoGroupSplitDep(rdd, _, itsSplit) => {
// Read them from the parent
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
index c5de6362a9..98da35763b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala
@@ -22,6 +22,7 @@ import java.io.{ObjectOutputStream, IOException}
import scala.collection.mutable
import scala.Some
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
/**
* Class that captures a coalesced RDD by essentially keeping track of parent partitions
@@ -68,7 +69,7 @@ case class CoalescedRDDPartition(
* @param maxPartitions number of desired partitions in the coalesced RDD
* @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance
*/
-class CoalescedRDD[T: ClassManifest](
+class CoalescedRDD[T: ClassTag](
@transient var prev: RDD[T],
maxPartitions: Int,
balanceSlack: Double = 0.10)
diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
index a4bec41752..688c310ee9 100644
--- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala
@@ -24,6 +24,8 @@ import org.apache.spark.partial.SumEvaluator
import org.apache.spark.util.StatCounter
import org.apache.spark.{TaskContext, Logging}
+import scala.collection.immutable.NumericRange
+
/**
* Extra functions available on RDDs of Doubles through an implicit conversion.
* Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
@@ -76,4 +78,129 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
val evaluator = new SumEvaluator(self.partitions.size, confidence)
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
}
+
+ /**
+ * Compute a histogram of the data using bucketCount number of buckets evenly
+ * spaced between the minimum and maximum of the RDD. For example if the min
+ * value is 0 and the max is 100 and there are two buckets the resulting
+ * buckets will be [0, 50) [50, 100]. bucketCount must be at least 1
+ * If the RDD contains infinity, NaN throws an exception
+ * If the elements in RDD do not vary (max == min) always returns a single bucket.
+ */
+ def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = {
+ // Compute the minimum and the maxium
+ val (max: Double, min: Double) = self.mapPartitions { items =>
+ Iterator(items.foldRight(Double.NegativeInfinity,
+ Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) =>
+ (x._1.max(e), x._2.min(e))))
+ }.reduce { (maxmin1, maxmin2) =>
+ (maxmin1._1.max(maxmin2._1), maxmin1._2.min(maxmin2._2))
+ }
+ if (min.isNaN || max.isNaN || max.isInfinity || min.isInfinity ) {
+ throw new UnsupportedOperationException(
+ "Histogram on either an empty RDD or RDD containing +/-infinity or NaN")
+ }
+ val increment = (max-min)/bucketCount.toDouble
+ val range = if (increment != 0) {
+ Range.Double.inclusive(min, max, increment)
+ } else {
+ List(min, min)
+ }
+ val buckets = range.toArray
+ (buckets, histogram(buckets, true))
+ }
+
+ /**
+ * Compute a histogram using the provided buckets. The buckets are all open
+ * to the left except for the last which is closed
+ * e.g. for the array
+ * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50]
+ * e.g 1<=x<10 , 10<=x<20, 20<=x<50
+ * And on the input of 1 and 50 we would have a histogram of 1, 0, 0
+ *
+ * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched
+ * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets
+ * to true.
+ * buckets must be sorted and not contain any duplicates.
+ * buckets array must be at least two elements
+ * All NaN entries are treated the same. If you have a NaN bucket it must be
+ * the maximum value of the last position and all NaN entries will be counted
+ * in that bucket.
+ */
+ def histogram(buckets: Array[Double], evenBuckets: Boolean = false): Array[Long] = {
+ if (buckets.length < 2) {
+ throw new IllegalArgumentException("buckets array must have at least two elements")
+ }
+ // The histogramPartition function computes the partail histogram for a given
+ // partition. The provided bucketFunction determines which bucket in the array
+ // to increment or returns None if there is no bucket. This is done so we can
+ // specialize for uniformly distributed buckets and save the O(log n) binary
+ // search cost.
+ def histogramPartition(bucketFunction: (Double) => Option[Int])(iter: Iterator[Double]):
+ Iterator[Array[Long]] = {
+ val counters = new Array[Long](buckets.length - 1)
+ while (iter.hasNext) {
+ bucketFunction(iter.next()) match {
+ case Some(x: Int) => {counters(x) += 1}
+ case _ => {}
+ }
+ }
+ Iterator(counters)
+ }
+ // Merge the counters.
+ def mergeCounters(a1: Array[Long], a2: Array[Long]): Array[Long] = {
+ a1.indices.foreach(i => a1(i) += a2(i))
+ a1
+ }
+ // Basic bucket function. This works using Java's built in Array
+ // binary search. Takes log(size(buckets))
+ def basicBucketFunction(e: Double): Option[Int] = {
+ val location = java.util.Arrays.binarySearch(buckets, e)
+ if (location < 0) {
+ // If the location is less than 0 then the insertion point in the array
+ // to keep it sorted is -location-1
+ val insertionPoint = -location-1
+ // If we have to insert before the first element or after the last one
+ // its out of bounds.
+ // We do this rather than buckets.lengthCompare(insertionPoint)
+ // because Array[Double] fails to override it (for now).
+ if (insertionPoint > 0 && insertionPoint < buckets.length) {
+ Some(insertionPoint-1)
+ } else {
+ None
+ }
+ } else if (location < buckets.length - 1) {
+ // Exact match, just insert here
+ Some(location)
+ } else {
+ // Exact match to the last element
+ Some(location - 1)
+ }
+ }
+ // Determine the bucket function in constant time. Requires that buckets are evenly spaced
+ def fastBucketFunction(min: Double, increment: Double, count: Int)(e: Double): Option[Int] = {
+ // If our input is not a number unless the increment is also NaN then we fail fast
+ if (e.isNaN()) {
+ return None
+ }
+ val bucketNumber = (e - min)/(increment)
+ // We do this rather than buckets.lengthCompare(bucketNumber)
+ // because Array[Double] fails to override it (for now).
+ if (bucketNumber > count || bucketNumber < 0) {
+ None
+ } else {
+ Some(bucketNumber.toInt.min(count - 1))
+ }
+ }
+ // Decide which bucket function to pass to histogramPartition. We decide here
+ // rather than having a general function so that the decission need only be made
+ // once rather than once per shard
+ val bucketFunction = if (evenBuckets) {
+ fastBucketFunction(buckets(0), buckets(1)-buckets(0), buckets.length-1) _
+ } else {
+ basicBucketFunction _
+ }
+ self.mapPartitions(histogramPartition(bucketFunction)).reduce(mergeCounters)
+ }
+
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
index c8900d1a93..a84e5f9fd8 100644
--- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala
@@ -17,13 +17,14 @@
package org.apache.spark.rdd
-import org.apache.spark.{SparkContext, SparkEnv, Partition, TaskContext}
+import scala.reflect.ClassTag
+import org.apache.spark.{Partition, SparkContext, TaskContext}
/**
* An RDD that is empty, i.e. has no element in it.
*/
-class EmptyRDD[T: ClassManifest](sc: SparkContext) extends RDD[T](sc, Nil) {
+class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) {
override def getPartitions: Array[Partition] = Array.empty
diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
index 5312dc0b59..e74c83b90b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala
@@ -18,8 +18,9 @@
package org.apache.spark.rdd
import org.apache.spark.{OneToOneDependency, Partition, TaskContext}
+import scala.reflect.ClassTag
-private[spark] class FilteredRDD[T: ClassManifest](
+private[spark] class FilteredRDD[T: ClassTag](
prev: RDD[T],
f: T => Boolean)
extends RDD[T](prev) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
index cbdf6d84c0..4d1878fc14 100644
--- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala
@@ -18,10 +18,11 @@
package org.apache.spark.rdd
import org.apache.spark.{Partition, TaskContext}
+import scala.reflect.ClassTag
private[spark]
-class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
+class FlatMappedRDD[U: ClassTag, T: ClassTag](
prev: RDD[T],
f: T => TraversableOnce[U])
extends RDD[U](prev) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
index 829545d7b0..1a694475f6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala
@@ -18,8 +18,9 @@
package org.apache.spark.rdd
import org.apache.spark.{Partition, TaskContext}
+import scala.reflect.ClassTag
-private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T])
+private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T])
extends RDD[Array[T]](prev) {
override def getPartitions: Array[Partition] = firstParent[T].partitions
diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index aca0146884..8df8718f3b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -19,6 +19,8 @@ package org.apache.spark.rdd
import java.sql.{Connection, ResultSet}
+import scala.reflect.ClassTag
+
import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
import org.apache.spark.util.NextIterator
@@ -45,7 +47,7 @@ private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) e
* This should only call getInt, getString, etc; the RDD takes care of calling next.
* The default maps a ResultSet to an array of Object.
*/
-class JdbcRDD[T: ClassManifest](
+class JdbcRDD[T: ClassTag](
sc: SparkContext,
getConnection: () => Connection,
sql: String,
diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
index 203179c4ea..db15baf503 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
@@ -18,20 +18,18 @@
package org.apache.spark.rdd
import org.apache.spark.{Partition, TaskContext}
+import scala.reflect.ClassTag
-
-private[spark]
-class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
+private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag](
prev: RDD[T],
- f: Iterator[T] => Iterator[U],
+ f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator)
preservesPartitioning: Boolean = false)
extends RDD[U](prev) {
- override val partitioner =
- if (preservesPartitioning) firstParent[T].partitioner else None
+ override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None
override def getPartitions: Array[Partition] = firstParent[T].partitions
override def compute(split: Partition, context: TaskContext) =
- f(firstParent[T].iterator(split, context))
+ f(context, split.index, firstParent[T].iterator(split, context))
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
index e8be1c4816..8d7c288593 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala
@@ -17,10 +17,12 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.spark.{Partition, TaskContext}
private[spark]
-class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U)
+class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U)
extends RDD[U](prev) {
override def getPartitions: Array[Partition] = firstParent[T].partitions
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index 2662d48c84..73d15b9082 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -76,7 +76,7 @@ class NewHadoopRDD[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 attemptId = newTaskAttemptID(jobtrackerId, id, isMap = true, split.index, 0)
val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId)
val format = inputFormatClass.newInstance
if (format.isInstanceOf[Configurable]) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
index 697be8b997..d5691f2267 100644
--- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala
@@ -17,7 +17,9 @@
package org.apache.spark.rdd
-import org.apache.spark.{RangePartitioner, Logging}
+import scala.reflect.ClassTag
+
+import org.apache.spark.{Logging, RangePartitioner}
/**
* Extra functions available on RDDs of (key, value) pairs where the key is sortable through
@@ -25,9 +27,9 @@ import org.apache.spark.{RangePartitioner, Logging}
* use these functions. They will work with any key type that has a `scala.math.Ordered`
* implementation.
*/
-class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest,
- V: ClassManifest,
- P <: Product2[K, V] : ClassManifest](
+class OrderedRDDFunctions[K <% Ordered[K]: ClassTag,
+ V: ClassTag,
+ P <: Product2[K, V] : ClassTag](
self: RDD[P])
extends Logging with Serializable {
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 53184a6b8a..2bf7c5b8d6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -25,6 +25,7 @@ import java.util.{HashMap => JHashMap}
import scala.collection.{mutable, Map}
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConversions._
+import scala.reflect.{ClassTag, classTag}
import org.apache.hadoop.mapred._
import org.apache.hadoop.io.compress.CompressionCodec
@@ -39,18 +40,21 @@ import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil
import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob}
import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter}
+import com.clearspring.analytics.stream.cardinality.HyperLogLog
+
import org.apache.spark._
import org.apache.spark.SparkContext._
import org.apache.spark.partial.{BoundedDouble, PartialResult}
import org.apache.spark.Aggregator
import org.apache.spark.Partitioner
import org.apache.spark.Partitioner.defaultPartitioner
+import org.apache.spark.util.SerializableHyperLogLog
/**
* Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
* Import `org.apache.spark.SparkContext._` at the top of your program to use these functions.
*/
-class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
+class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)])
extends Logging
with SparkHadoopMapReduceUtil
with Serializable {
@@ -207,6 +211,45 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
}
/**
+ * Return approximate number of distinct values for each key in this RDD.
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. Uses the provided
+ * Partitioner to partition the output RDD.
+ */
+ def countApproxDistinctByKey(relativeSD: Double, partitioner: Partitioner): RDD[(K, Long)] = {
+ val createHLL = (v: V) => new SerializableHyperLogLog(new HyperLogLog(relativeSD)).add(v)
+ val mergeValueHLL = (hll: SerializableHyperLogLog, v: V) => hll.add(v)
+ val mergeHLL = (h1: SerializableHyperLogLog, h2: SerializableHyperLogLog) => h1.merge(h2)
+
+ combineByKey(createHLL, mergeValueHLL, mergeHLL, partitioner).mapValues(_.value.cardinality())
+ }
+
+ /**
+ * Return approximate number of distinct values for each key in this RDD.
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. HashPartitions the
+ * output RDD into numPartitions.
+ *
+ */
+ def countApproxDistinctByKey(relativeSD: Double, numPartitions: Int): RDD[(K, Long)] = {
+ countApproxDistinctByKey(relativeSD, new HashPartitioner(numPartitions))
+ }
+
+ /**
+ * Return approximate number of distinct values for each key this RDD.
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. The default value of
+ * relativeSD is 0.05. Hash-partitions the output RDD using the existing partitioner/parallelism
+ * level.
+ */
+ def countApproxDistinctByKey(relativeSD: Double = 0.05): RDD[(K, Long)] = {
+ countApproxDistinctByKey(relativeSD, defaultPartitioner(self))
+ }
+
+ /**
* Merge the values for each key using an associative reduce function. This will also perform
* the merging locally on each mapper before sending results to a reducer, similarly to a
* "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.
@@ -415,7 +458,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](Seq(self, other), partitioner)
- val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
+ val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag)
prfs.mapValues { case Seq(vs, ws) =>
(vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
}
@@ -431,7 +474,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner)
- val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
+ val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classTag[K], ClassTags.seqSeqClassTag)
prfs.mapValues { case Seq(vs, w1s, w2s) =>
(vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]])
}
@@ -488,15 +531,15 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
* Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
* RDD will be <= us.
*/
- def subtractByKey[W: ClassManifest](other: RDD[(K, W)]): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] =
subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size)))
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
- def subtractByKey[W: ClassManifest](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] =
subtractByKey(other, new HashPartitioner(numPartitions))
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
- def subtractByKey[W: ClassManifest](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] =
+ def subtractByKey[W: ClassTag](other: RDD[(K, W)], p: Partitioner): RDD[(K, V)] =
new SubtractedRDD[K, V, W](self, other, p)
/**
@@ -525,8 +568,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
* Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class
* supporting the key and value types K and V in this RDD.
*/
- def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) {
- saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) {
+ saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -535,16 +578,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
* supplied codec.
*/
def saveAsHadoopFile[F <: OutputFormat[K, V]](
- path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassManifest[F]) {
- saveAsHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]], codec)
+ path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) {
+ saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]], codec)
}
/**
* Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
* (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
*/
- def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassManifest[F]) {
- saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) {
+ saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -570,7 +613,8 @@ 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.partitionId, attemptNumber)
+ val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId,
+ attemptNumber)
val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
val format = outputFormatClass.newInstance
val committer = format.getOutputCommitter(hadoopContext)
@@ -589,13 +633,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
* however we're only going to use this local OutputCommitter for
* setupJob/commitJob, so we just use a dummy "map" task.
*/
- val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, true, 0, 0)
+ val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0)
val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId)
val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext)
jobCommitter.setupJob(jobTaskContext)
val count = self.context.runJob(self, writeShard _).sum
jobCommitter.commitJob(jobTaskContext)
- jobCommitter.cleanupJob(jobTaskContext)
}
/**
@@ -685,7 +728,6 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
self.context.runJob(self, writeToFile _)
writer.commitJob()
- writer.cleanup()
}
/**
@@ -698,11 +740,11 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](self: RDD[(K, V)])
*/
def values: RDD[V] = self.map(_._2)
- private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure
+ private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass
- private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure
+ private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass
}
-private[spark] object Manifests {
- val seqSeqManifest = classManifest[Seq[Seq[_]]]
+private[spark] object ClassTags {
+ val seqSeqClassTag = classTag[Seq[Seq[_]]]
}
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 cd96250389..09d0a8189d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala
@@ -20,13 +20,15 @@ package org.apache.spark.rdd
import scala.collection.immutable.NumericRange
import scala.collection.mutable.ArrayBuffer
import scala.collection.Map
+import scala.reflect.ClassTag
+
import org.apache.spark._
import java.io._
import scala.Serializable
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.util.Utils
-private[spark] class ParallelCollectionPartition[T: ClassManifest](
+private[spark] class ParallelCollectionPartition[T: ClassTag](
var rddId: Long,
var slice: Int,
var values: Seq[T])
@@ -78,7 +80,7 @@ private[spark] class ParallelCollectionPartition[T: ClassManifest](
}
}
-private[spark] class ParallelCollectionRDD[T: ClassManifest](
+private[spark] class ParallelCollectionRDD[T: ClassTag](
@transient sc: SparkContext,
@transient data: Seq[T],
numSlices: Int,
@@ -109,7 +111,7 @@ private object ParallelCollectionRDD {
* collections specially, encoding the slices as other Ranges to minimize memory cost. This makes
* it efficient to run Spark over RDDs representing large sets of numbers.
*/
- def slice[T: ClassManifest](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = {
+ def slice[T: ClassTag](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = {
if (numSlices < 1) {
throw new IllegalArgumentException("Positive number of slices required")
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
index 574dd4233f..ea8885b36e 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala
@@ -17,6 +17,8 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.spark.{NarrowDependency, SparkEnv, Partition, TaskContext}
@@ -49,7 +51,7 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo
* and the execution DAG has a filter on the key, we can avoid launching tasks
* on partitions that don't have the range covering the key.
*/
-class PartitionPruningRDD[T: ClassManifest](
+class PartitionPruningRDD[T: ClassTag](
@transient prev: RDD[T],
@transient partitionFilterFunc: Int => Boolean)
extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) {
@@ -69,6 +71,6 @@ object PartitionPruningRDD {
* when its type T is not known at compile time.
*/
def create[T](rdd: RDD[T], partitionFilterFunc: Int => Boolean) = {
- new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassManifest)
+ new PartitionPruningRDD[T](rdd, partitionFilterFunc)(rdd.elementClassTag)
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala
new file mode 100644
index 0000000000..4c625d062e
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala
@@ -0,0 +1,110 @@
+/*
+ * 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 scala.reflect.ClassTag
+import java.io.{ObjectOutputStream, IOException}
+import org.apache.spark.{TaskContext, OneToOneDependency, SparkContext, Partition}
+
+
+/**
+ * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of corresponding partitions
+ * of parent RDDs.
+ */
+private[spark]
+class PartitionerAwareUnionRDDPartition(
+ @transient val rdds: Seq[RDD[_]],
+ val idx: Int
+ ) extends Partition {
+ var parents = rdds.map(_.partitions(idx)).toArray
+
+ override val index = idx
+ override def hashCode(): Int = idx
+
+ @throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ // Update the reference to parent partition at the time of task serialization
+ parents = rdds.map(_.partitions(index)).toArray
+ oos.defaultWriteObject()
+ }
+}
+
+/**
+ * Class representing an RDD that can take multiple RDDs partitioned by the same partitioner and
+ * unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each
+ * will be unified to a single RDD with p partitions and the same partitioner. The preferred
+ * location for each partition of the unified RDD will be the most common preferred location
+ * of the corresponding partitions of the parent RDDs. For example, location of partition 0
+ * of the unified RDD will be where most of partition 0 of the parent RDDs are located.
+ */
+private[spark]
+class PartitionerAwareUnionRDD[T: ClassTag](
+ sc: SparkContext,
+ var rdds: Seq[RDD[T]]
+ ) extends RDD[T](sc, rdds.map(x => new OneToOneDependency(x))) {
+ require(rdds.length > 0)
+ require(rdds.flatMap(_.partitioner).toSet.size == 1,
+ "Parent RDDs have different partitioners: " + rdds.flatMap(_.partitioner))
+
+ override val partitioner = rdds.head.partitioner
+
+ override def getPartitions: Array[Partition] = {
+ val numPartitions = partitioner.get.numPartitions
+ (0 until numPartitions).map(index => {
+ new PartitionerAwareUnionRDDPartition(rdds, index)
+ }).toArray
+ }
+
+ // Get the location where most of the partitions of parent RDDs are located
+ override def getPreferredLocations(s: Partition): Seq[String] = {
+ logDebug("Finding preferred location for " + this + ", partition " + s.index)
+ val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents
+ val locations = rdds.zip(parentPartitions).flatMap {
+ case (rdd, part) => {
+ val parentLocations = currPrefLocs(rdd, part)
+ logDebug("Location of " + rdd + " partition " + part.index + " = " + parentLocations)
+ parentLocations
+ }
+ }
+ val location = if (locations.isEmpty) {
+ None
+ } else {
+ // Find the location that maximum number of parent partitions prefer
+ Some(locations.groupBy(x => x).maxBy(_._2.length)._1)
+ }
+ logDebug("Selected location for " + this + ", partition " + s.index + " = " + location)
+ location.toSeq
+ }
+
+ override def compute(s: Partition, context: TaskContext): Iterator[T] = {
+ val parentPartitions = s.asInstanceOf[PartitionerAwareUnionRDDPartition].parents
+ rdds.zip(parentPartitions).iterator.flatMap {
+ case (rdd, p) => rdd.iterator(p, context)
+ }
+ }
+
+ override def clearDependencies() {
+ super.clearDependencies()
+ rdds = null
+ }
+
+ // Get the *current* preferred locations from the DAGScheduler (as opposed to the static ones)
+ private def currPrefLocs(rdd: RDD[_], part: Partition): Seq[String] = {
+ rdd.context.getPreferredLocs(rdd, part.index).map(tl => tl.host)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
index d5304ab0ae..1dbbe39898 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -24,6 +24,7 @@ import scala.collection.Map
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
+import scala.reflect.ClassTag
import org.apache.spark.{SparkEnv, Partition, TaskContext}
import org.apache.spark.broadcast.Broadcast
@@ -33,7 +34,7 @@ import org.apache.spark.broadcast.Broadcast
* An RDD that pipes the contents of each parent partition through an external command
* (printing them one per line) and returns the output as a collection of strings.
*/
-class PipedRDD[T: ClassManifest](
+class PipedRDD[T: ClassTag](
prev: RDD[T],
command: Seq[String],
envVars: Map[String, String],
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 7623c44d88..3f41b66279 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -23,6 +23,9 @@ import scala.collection.Map
import scala.collection.JavaConversions.mapAsScalaMap
import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashMap
+import scala.reflect.{classTag, ClassTag}
+
import org.apache.hadoop.io.BytesWritable
import org.apache.hadoop.io.compress.CompressionCodec
import org.apache.hadoop.io.NullWritable
@@ -30,6 +33,7 @@ import org.apache.hadoop.io.Text
import org.apache.hadoop.mapred.TextOutputFormat
import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
+import com.clearspring.analytics.stream.cardinality.HyperLogLog
import org.apache.spark.Partitioner._
import org.apache.spark.api.java.JavaRDD
@@ -38,7 +42,7 @@ import org.apache.spark.partial.CountEvaluator
import org.apache.spark.partial.GroupedCountEvaluator
import org.apache.spark.partial.PartialResult
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.{Utils, BoundedPriorityQueue}
+import org.apache.spark.util.{Utils, BoundedPriorityQueue, SerializableHyperLogLog}
import org.apache.spark.SparkContext._
import org.apache.spark._
@@ -69,7 +73,7 @@ import org.apache.spark._
* [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details
* on RDD internals.
*/
-abstract class RDD[T: ClassManifest](
+abstract class RDD[T: ClassTag](
@transient private var sc: SparkContext,
@transient private var deps: Seq[Dependency[_]]
) extends Serializable with Logging {
@@ -78,6 +82,7 @@ abstract class RDD[T: ClassManifest](
def this(@transient oneParent: RDD[_]) =
this(oneParent.context , List(new OneToOneDependency(oneParent)))
+ private[spark] def conf = sc.conf
// =======================================================================
// Methods that should be implemented by subclasses of RDD
// =======================================================================
@@ -101,7 +106,7 @@ abstract class RDD[T: ClassManifest](
protected def getPreferredLocations(split: Partition): Seq[String] = Nil
/** Optionally overridden by subclasses to specify how they are partitioned. */
- val partitioner: Option[Partitioner] = None
+ @transient val partitioner: Option[Partitioner] = None
// =======================================================================
// Methods and fields available on all RDDs
@@ -114,7 +119,7 @@ abstract class RDD[T: ClassManifest](
val id: Int = sc.newRddId()
/** A friendly name for this RDD */
- var name: String = null
+ @transient var name: String = null
/** Assign a name to this RDD */
def setName(_name: String) = {
@@ -123,7 +128,7 @@ abstract class RDD[T: ClassManifest](
}
/** User-defined generator of this RDD*/
- var generator = Utils.getCallSiteInfo.firstUserClass
+ @transient var generator = Utils.getCallSiteInfo.firstUserClass
/** Reset generator*/
def setGenerator(_generator: String) = {
@@ -243,13 +248,13 @@ abstract class RDD[T: ClassManifest](
/**
* Return a new RDD by applying a function to all elements of this RDD.
*/
- def map[U: ClassManifest](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f))
+ def map[U: ClassTag](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f))
/**
* Return a new RDD by first applying a function to all elements of this
* RDD, and then flattening the results.
*/
- def flatMap[U: ClassManifest](f: T => TraversableOnce[U]): RDD[U] =
+ def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] =
new FlatMappedRDD(this, sc.clean(f))
/**
@@ -374,25 +379,25 @@ abstract class RDD[T: ClassManifest](
* Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of
* elements (a, b) where a is in `this` and b is in `other`.
*/
- def cartesian[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other)
+ def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other)
/**
* Return an RDD of grouped items.
*/
- def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] =
+ def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] =
groupBy[K](f, defaultPartitioner(this))
/**
* Return an RDD of grouped elements. Each group consists of a key and a sequence of elements
* mapping to that key.
*/
- def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] =
+ def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] =
groupBy(f, new HashPartitioner(numPartitions))
/**
* Return an RDD of grouped items.
*/
- def groupBy[K: ClassManifest](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = {
+ def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = {
val cleanF = sc.clean(f)
this.map(t => (cleanF(t), t)).groupByKey(p)
}
@@ -408,7 +413,6 @@ abstract class RDD[T: ClassManifest](
def pipe(command: String, env: Map[String, String]): RDD[String] =
new PipedRDD(this, command, env)
-
/**
* Return an RDD created by piping elements to a forked external process.
* The print behavior can be customized by providing two functions.
@@ -440,29 +444,31 @@ abstract class RDD[T: ClassManifest](
/**
* Return a new RDD by applying a function to each partition of this RDD.
*/
- def mapPartitions[U: ClassManifest](
+ def mapPartitions[U: ClassTag](
f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
- new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning)
+ val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(iter)
+ new MapPartitionsRDD(this, sc.clean(func), 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](
+ def mapPartitionsWithIndex[U: ClassTag](
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)
+ val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter)
+ new MapPartitionsRDD(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](
+ def mapPartitionsWithContext[U: ClassTag](
f: (TaskContext, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean = false): RDD[U] = {
- new MapPartitionsWithContextRDD(this, sc.clean(f), preservesPartitioning)
+ val func = (context: TaskContext, index: Int, iter: Iterator[T]) => f(context, iter)
+ new MapPartitionsRDD(this, sc.clean(func), preservesPartitioning)
}
/**
@@ -470,7 +476,7 @@ abstract class RDD[T: ClassManifest](
* of the original partition.
*/
@deprecated("use mapPartitionsWithIndex", "0.7.0")
- def mapPartitionsWithSplit[U: ClassManifest](
+ def mapPartitionsWithSplit[U: ClassTag](
f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = {
mapPartitionsWithIndex(f, preservesPartitioning)
}
@@ -480,14 +486,13 @@ abstract class RDD[T: ClassManifest](
* additional parameter is produced by constructA, which is called in each
* partition with the index of that partition.
*/
- def mapWith[A: ClassManifest, U: ClassManifest]
+ def mapWith[A: ClassTag, U: ClassTag]
(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)
+ mapPartitionsWithIndex((index, iter) => {
+ val a = constructA(index)
iter.map(t => f(t, a))
- }
- new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning)
+ }, preservesPartitioning)
}
/**
@@ -495,14 +500,13 @@ 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]
+ def flatMapWith[A: ClassTag, U: ClassTag]
(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)
+ mapPartitionsWithIndex((index, iter) => {
+ val a = constructA(index)
iter.flatMap(t => f(t, a))
- }
- new MapPartitionsWithContextRDD(this, sc.clean(iterF _), preservesPartitioning)
+ }, preservesPartitioning)
}
/**
@@ -510,12 +514,11 @@ 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(context: TaskContext, iter: Iterator[T]): Iterator[T] = {
- val a = constructA(context.partitionId)
+ def foreachWith[A: ClassTag](constructA: Int => A)(f: (T, A) => Unit) {
+ mapPartitionsWithIndex { (index, iter) =>
+ val a = constructA(index)
iter.map(t => {f(t, a); t})
- }
- new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true).foreach(_ => {})
+ }.foreach(_ => {})
}
/**
@@ -523,12 +526,11 @@ 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(context: TaskContext, iter: Iterator[T]): Iterator[T] = {
- val a = constructA(context.partitionId)
+ def filterWith[A: ClassTag](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = {
+ mapPartitionsWithIndex((index, iter) => {
+ val a = constructA(index)
iter.filter(t => p(t, a))
- }
- new MapPartitionsWithContextRDD(this, sc.clean(iterF _), true)
+ }, preservesPartitioning = true)
}
/**
@@ -537,7 +539,7 @@ abstract class RDD[T: ClassManifest](
* partitions* and the *same number of elements in each partition* (e.g. one was made through
* a map on the other).
*/
- def zip[U: ClassManifest](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other)
+ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new ZippedRDD(sc, this, other)
/**
* Zip this RDD's partitions with one (or more) RDD(s) and return a new RDD by
@@ -545,32 +547,27 @@ abstract class RDD[T: ClassManifest](
* *same number of partitions*, but does *not* require them to have the same number
* of elements in each partition.
*/
- def zipPartitions[B: ClassManifest, V: ClassManifest]
- (rdd2: RDD[B], preservesPartitioning: Boolean)
- (f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
- new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, preservesPartitioning)
-
- def zipPartitions[B: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, V: ClassTag]
(rdd2: RDD[B])
(f: (Iterator[T], Iterator[B]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD2(sc, sc.clean(f), this, rdd2, false)
- def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], preservesPartitioning: Boolean)
(f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, preservesPartitioning)
- def zipPartitions[B: ClassManifest, C: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, C: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C])
(f: (Iterator[T], Iterator[B], Iterator[C]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD3(sc, sc.clean(f), this, rdd2, rdd3, false)
- def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D], preservesPartitioning: Boolean)
(f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, preservesPartitioning)
- def zipPartitions[B: ClassManifest, C: ClassManifest, D: ClassManifest, V: ClassManifest]
+ def zipPartitions[B: ClassTag, C: ClassTag, D: ClassTag, V: ClassTag]
(rdd2: RDD[B], rdd3: RDD[C], rdd4: RDD[D])
(f: (Iterator[T], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V]): RDD[V] =
new ZippedPartitionsRDD4(sc, sc.clean(f), this, rdd2, rdd3, rdd4, false)
@@ -608,7 +605,7 @@ abstract class RDD[T: ClassManifest](
/**
* Return an RDD that contains all matching values by applying `f`.
*/
- def collect[U: ClassManifest](f: PartialFunction[T, U]): RDD[U] = {
+ def collect[U: ClassTag](f: PartialFunction[T, U]): RDD[U] = {
filter(f.isDefinedAt).map(f)
}
@@ -698,7 +695,7 @@ abstract class RDD[T: ClassManifest](
* allowed to modify and return their first argument instead of creating a new U to avoid memory
* allocation.
*/
- def aggregate[U: ClassManifest](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = {
+ def aggregate[U: ClassTag](zeroValue: U)(seqOp: (U, T) => U, combOp: (U, U) => U): U = {
// Clone the zero value since we will also be serializing it as part of tasks
var jobResult = Utils.clone(zeroValue, sc.env.closureSerializer.newInstance())
val cleanSeqOp = sc.clean(seqOp)
@@ -747,7 +744,7 @@ abstract class RDD[T: ClassManifest](
* combine step happens locally on the master, equivalent to running a single reduce task.
*/
def countByValue(): Map[T, Long] = {
- if (elementClassManifest.erasure.isArray) {
+ if (elementClassTag.runtimeClass.isArray) {
throw new SparkException("countByValue() does not support arrays")
}
// TODO: This should perhaps be distributed by default.
@@ -778,7 +775,7 @@ abstract class RDD[T: ClassManifest](
timeout: Long,
confidence: Double = 0.95
): PartialResult[Map[T, BoundedDouble]] = {
- if (elementClassManifest.erasure.isArray) {
+ if (elementClassTag.runtimeClass.isArray) {
throw new SparkException("countByValueApprox() does not support arrays")
}
val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) =>
@@ -794,6 +791,19 @@ abstract class RDD[T: ClassManifest](
}
/**
+ * Return approximate number of distinct elements in the RDD.
+ *
+ * The accuracy of approximation can be controlled through the relative standard deviation
+ * (relativeSD) parameter, which also controls the amount of memory used. Lower values result in
+ * more accurate counts but increase the memory footprint and vise versa. The default value of
+ * relativeSD is 0.05.
+ */
+ def countApproxDistinct(relativeSD: Double = 0.05): Long = {
+ val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD))
+ aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality()
+ }
+
+ /**
* 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.
@@ -943,14 +953,14 @@ abstract class RDD[T: ClassManifest](
private var storageLevel: StorageLevel = StorageLevel.NONE
/** Record user function generating this RDD. */
- private[spark] val origin = Utils.formatSparkCallSite
+ @transient private[spark] val origin = sc.getCallSite
- private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
+ private[spark] def elementClassTag: ClassTag[T] = classTag[T]
private[spark] var checkpointData: Option[RDDCheckpointData[T]] = None
/** Returns the first parent RDD */
- protected[spark] def firstParent[U: ClassManifest] = {
+ protected[spark] def firstParent[U: ClassTag] = {
dependencies.head.rdd.asInstanceOf[RDD[U]]
}
@@ -958,7 +968,7 @@ abstract class RDD[T: ClassManifest](
def context = sc
// Avoid handling doCheckpoint multiple times to prevent excessive recursion
- private var doCheckpointCalled = false
+ @transient private var doCheckpointCalled = false
/**
* Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler
@@ -1012,7 +1022,7 @@ abstract class RDD[T: ClassManifest](
origin)
def toJavaRDD() : JavaRDD[T] = {
- new JavaRDD(this)(elementClassManifest)
+ new JavaRDD(this)(elementClassTag)
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
index 6009a41570..bc688110f4 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala
@@ -17,10 +17,12 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
+
import org.apache.hadoop.fs.Path
import org.apache.hadoop.conf.Configuration
-import org.apache.spark.{Partition, SparkException, Logging}
+import org.apache.spark.{SerializableWritable, Partition, SparkException, Logging}
import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask}
/**
@@ -38,7 +40,7 @@ private[spark] object CheckpointState extends Enumeration {
* manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations
* of the checkpointed RDD.
*/
-private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
+private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T])
extends Logging with Serializable {
import CheckpointState._
@@ -83,14 +85,21 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
// Create the output path for the checkpoint
val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id)
- val fs = path.getFileSystem(new Configuration())
+ val fs = path.getFileSystem(rdd.context.hadoopConfiguration)
if (!fs.mkdirs(path)) {
throw new SparkException("Failed to create checkpoint path " + path)
}
// Save to file, and reload it as an RDD
- rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString) _)
+ val broadcastedConf = rdd.context.broadcast(
+ new SerializableWritable(rdd.context.hadoopConfiguration))
+ rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path.toString, broadcastedConf) _)
val newRDD = new CheckpointRDD[T](rdd.context, path.toString)
+ if (newRDD.partitions.size != rdd.partitions.size) {
+ throw new SparkException(
+ "Checkpoint RDD " + newRDD + "("+ newRDD.partitions.size + ") has different " +
+ "number of partitions than original RDD " + rdd + "(" + rdd.partitions.size + ")")
+ }
// Change the dependencies and partitions of the RDD
RDDCheckpointData.synchronized {
@@ -99,8 +108,8 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions
cpState = Checkpointed
RDDCheckpointData.clearTaskCaches()
- logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id)
}
+ logInfo("Done checkpointing RDD " + rdd.id + " to " + path + ", new parent is RDD " + newRDD.id)
}
// Get preferred location of a split after checkpointing
diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
index 2c5253ae30..d433670cc2 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
@@ -17,6 +17,7 @@
package org.apache.spark.rdd
+import scala.reflect.ClassTag
import java.util.Random
import cern.jet.random.Poisson
@@ -29,9 +30,9 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition
override val index: Int = prev.index
}
-class SampledRDD[T: ClassManifest](
+class SampledRDD[T: ClassTag](
prev: RDD[T],
- withReplacement: Boolean,
+ withReplacement: Boolean,
frac: Double,
seed: Int)
extends RDD[T](prev) {
diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
index 5fe4676029..2d1bd5b481 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala
@@ -14,9 +14,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.spark.rdd
+import scala.reflect.{ ClassTag, classTag}
+
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapred.SequenceFileOutputFormat
import org.apache.hadoop.io.compress.CompressionCodec
@@ -32,15 +33,15 @@ import org.apache.spark.Logging
*
* Import `org.apache.spark.SparkContext._` at the top of their program to use these functions.
*/
-class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest](
+class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag](
self: RDD[(K, V)])
extends Logging
with Serializable {
- private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = {
+ private def getWritableClass[T <% Writable: ClassTag](): Class[_ <: Writable] = {
val c = {
- if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) {
- classManifest[T].erasure
+ if (classOf[Writable].isAssignableFrom(classTag[T].runtimeClass)) {
+ classTag[T].runtimeClass
} else {
// We get the type of the Writable class by looking at the apply method which converts
// from T to Writable. Since we have two apply methods we filter out the one which
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 a5d751a7bd..0ccb309d0d 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -17,8 +17,10 @@
package org.apache.spark.rdd
-import org.apache.spark.{Dependency, Partitioner, SparkEnv, ShuffleDependency, Partition, TaskContext}
+import scala.reflect.ClassTag
+import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency,
+ SparkEnv, TaskContext}
private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
override val index = idx
@@ -32,7 +34,7 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
* @tparam K the key class.
* @tparam V the value class.
*/
-class ShuffledRDD[K, V, P <: Product2[K, V] : ClassManifest](
+class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag](
@transient var prev: RDD[P],
part: Partitioner)
extends RDD[P](prev.context, Nil) {
@@ -57,7 +59,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,
- SparkEnv.get.serializerManager.get(serializerClass))
+ SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf))
}
override def clearDependencies() {
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 7af4d803e7..4f90c7d3d6 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala
@@ -18,8 +18,11 @@
package org.apache.spark.rdd
import java.util.{HashMap => JHashMap}
+
import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
import org.apache.spark.Partitioner
import org.apache.spark.Dependency
import org.apache.spark.TaskContext
@@ -45,7 +48,7 @@ import org.apache.spark.OneToOneDependency
* you can use `rdd1`'s partitioner/partition size and not worry about running
* out of memory because of the size of `rdd2`.
*/
-private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassManifest](
+private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag](
@transient var rdd1: RDD[_ <: Product2[K, V]],
@transient var rdd2: RDD[_ <: Product2[K, W]],
part: Partitioner)
@@ -90,7 +93,7 @@ private[spark] class SubtractedRDD[K: ClassManifest, V: ClassManifest, W: ClassM
override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = {
val partition = p.asInstanceOf[CoGroupPartition]
- val serializer = SparkEnv.get.serializerManager.get(serializerClass)
+ val serializer = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)
val map = new JHashMap[K, ArrayBuffer[V]]
def getSeq(k: K): ArrayBuffer[V] = {
val seq = map.get(k)
diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
index ae8a9f36a6..08a41ac558 100644
--- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala
@@ -18,10 +18,13 @@
package org.apache.spark.rdd
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
import org.apache.spark.{Dependency, RangeDependency, SparkContext, Partition, TaskContext}
+
import java.io.{ObjectOutputStream, IOException}
-private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
+private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int)
extends Partition {
var split: Partition = rdd.partitions(splitIndex)
@@ -40,7 +43,7 @@ private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], spl
}
}
-class UnionRDD[T: ClassManifest](
+class UnionRDD[T: ClassTag](
sc: SparkContext,
@transient var rdds: Seq[RDD[T]])
extends RDD[T](sc, Nil) { // Nil since we implement getDependencies
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
index a97d2a01c8..83be3c6eb4 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala
@@ -19,6 +19,7 @@ package org.apache.spark.rdd
import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext}
import java.io.{ObjectOutputStream, IOException}
+import scala.reflect.ClassTag
private[spark] class ZippedPartitionsPartition(
idx: Int,
@@ -38,7 +39,7 @@ private[spark] class ZippedPartitionsPartition(
}
}
-abstract class ZippedPartitionsBaseRDD[V: ClassManifest](
+abstract class ZippedPartitionsBaseRDD[V: ClassTag](
sc: SparkContext,
var rdds: Seq[RDD[_]],
preservesPartitioning: Boolean = false)
@@ -71,7 +72,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassManifest](
}
}
-class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest](
+class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag](
sc: SparkContext,
f: (Iterator[A], Iterator[B]) => Iterator[V],
var rdd1: RDD[A],
@@ -92,7 +93,7 @@ class ZippedPartitionsRDD2[A: ClassManifest, B: ClassManifest, V: ClassManifest]
}
class ZippedPartitionsRDD3
- [A: ClassManifest, B: ClassManifest, C: ClassManifest, V: ClassManifest](
+ [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag](
sc: SparkContext,
f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V],
var rdd1: RDD[A],
@@ -117,7 +118,7 @@ class ZippedPartitionsRDD3
}
class ZippedPartitionsRDD4
- [A: ClassManifest, B: ClassManifest, C: ClassManifest, D:ClassManifest, V: ClassManifest](
+ [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag](
sc: SparkContext,
f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V],
var rdd1: RDD[A],
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
index 567b67dfee..fb5b070c18 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala
@@ -18,10 +18,12 @@
package org.apache.spark.rdd
import org.apache.spark.{OneToOneDependency, SparkContext, Partition, TaskContext}
+
import java.io.{ObjectOutputStream, IOException}
+import scala.reflect.ClassTag
-private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest](
+private[spark] class ZippedPartition[T: ClassTag, U: ClassTag](
idx: Int,
@transient rdd1: RDD[T],
@transient rdd2: RDD[U]
@@ -42,7 +44,7 @@ private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest](
}
}
-class ZippedRDD[T: ClassManifest, U: ClassManifest](
+class ZippedRDD[T: ClassTag, U: ClassTag](
sc: SparkContext,
var rdd1: RDD[T],
var rdd2: RDD[U])
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 42bb3884c8..043e01dbfb 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -21,9 +21,11 @@ import java.io.NotSerializableException
import java.util.Properties
import java.util.concurrent.atomic.AtomicInteger
-import akka.actor._
-import akka.util.duration._
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
+
+import akka.actor._
import org.apache.spark._
import org.apache.spark.rdd.RDD
@@ -104,36 +106,16 @@ class DAGScheduler(
// The time, in millis, to wait for fetch failure events to stop coming in after one is detected;
// this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one
// as more failure events come in
- val RESUBMIT_TIMEOUT = 50L
+ val RESUBMIT_TIMEOUT = 50.milliseconds
// The time, in millis, to wake up between polls of the completion queue in order to potentially
// resubmit failed stages
val POLL_TIMEOUT = 10L
- private val eventProcessActor: ActorRef = env.actorSystem.actorOf(Props(new Actor {
- override def preStart() {
- context.system.scheduler.schedule(RESUBMIT_TIMEOUT milliseconds, RESUBMIT_TIMEOUT milliseconds) {
- if (failed.size > 0) {
- resubmitFailedStages()
- }
- }
- }
-
- /**
- * The main event loop of the DAG scheduler, which waits for new-job / task-finished / failure
- * events and responds by launching tasks. This runs in a dedicated thread and receives events
- * via the eventQueue.
- */
- def receive = {
- case event: DAGSchedulerEvent =>
- logDebug("Got event of type " + event.getClass.getName)
+ // Warns the user if a stage contains a task with size greater than this value (in KB)
+ val TASK_SIZE_TO_WARN = 100
- if (!processEvent(event))
- submitWaitingStages()
- else
- context.stop(self)
- }
- }))
+ private var eventProcessActor: ActorRef = _
private[scheduler] val nextJobId = new AtomicInteger(0)
@@ -141,9 +123,13 @@ class DAGScheduler(
private val nextStageId = new AtomicInteger(0)
- private val stageIdToStage = new TimeStampedHashMap[Int, Stage]
+ private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]]
- private val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
+ private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]]
+
+ private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage]
+
+ private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo]
@@ -166,13 +152,66 @@ class DAGScheduler(
val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done
val running = new HashSet[Stage] // Stages we are running right now
val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures
- val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] // Missing tasks from each stage
+ // Missing tasks from each stage
+ val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]]
var lastFetchFailureTime: Long = 0 // Used to wait a bit to avoid repeated resubmits
val activeJobs = new HashSet[ActiveJob]
val resultStageToJob = new HashMap[Stage, ActiveJob]
- val metadataCleaner = new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup)
+ val metadataCleaner = new MetadataCleaner(
+ MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf)
+
+ /**
+ * Starts the event processing actor. The actor has two responsibilities:
+ *
+ * 1. Waits for events like job submission, task finished, task failure etc., and calls
+ * [[org.apache.spark.scheduler.DAGScheduler.processEvent()]] to process them.
+ * 2. Schedules a periodical task to resubmit failed stages.
+ *
+ * NOTE: the actor cannot be started in the constructor, because the periodical task references
+ * some internal states of the enclosing [[org.apache.spark.scheduler.DAGScheduler]] object, thus
+ * cannot be scheduled until the [[org.apache.spark.scheduler.DAGScheduler]] is fully constructed.
+ */
+ def start() {
+ eventProcessActor = env.actorSystem.actorOf(Props(new Actor {
+ /**
+ * A handle to the periodical task, used to cancel the task when the actor is stopped.
+ */
+ var resubmissionTask: Cancellable = _
+
+ override def preStart() {
+ import context.dispatcher
+ /**
+ * A message is sent to the actor itself periodically to remind the actor to resubmit failed
+ * stages. In this way, stage resubmission can be done within the same thread context of
+ * other event processing logic to avoid unnecessary synchronization overhead.
+ */
+ resubmissionTask = context.system.scheduler.schedule(
+ RESUBMIT_TIMEOUT, RESUBMIT_TIMEOUT, self, ResubmitFailedStages)
+ }
+
+ /**
+ * The main event loop of the DAG scheduler.
+ */
+ def receive = {
+ case event: DAGSchedulerEvent =>
+ logDebug("Got event of type " + event.getClass.getName)
+
+ /**
+ * All events are forwarded to `processEvent()`, so that the event processing logic can
+ * easily tested without starting a dedicated actor. Please refer to `DAGSchedulerSuite`
+ * for details.
+ */
+ if (!processEvent(event)) {
+ submitWaitingStages()
+ } else {
+ resubmissionTask.cancel()
+ context.stop(self)
+ }
+ }
+ }))
+ }
def addSparkListener(listener: SparkListener) {
listenerBus.addListener(listener)
@@ -202,16 +241,18 @@ class DAGScheduler(
shuffleToMapStage.get(shuffleDep.shuffleId) match {
case Some(stage) => stage
case None =>
- val stage = newStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, Some(shuffleDep), jobId)
+ val stage =
+ newOrUsedStage(shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId)
shuffleToMapStage(shuffleDep.shuffleId) = stage
stage
}
}
/**
- * Create a Stage for the given RDD, either as a shuffle map stage (for a ShuffleDependency) or
- * as a result stage for the final RDD used directly in an action. The stage will also be
- * associated with the provided jobId.
+ * Create a Stage -- either directly for use as a result stage, or as part of the (re)-creation
+ * of a shuffle map stage in newOrUsedStage. The stage will be associated with the provided
+ * jobId. Production of shuffle map stages should always use newOrUsedStage, not newStage
+ * directly.
*/
private def newStage(
rdd: RDD[_],
@@ -221,21 +262,45 @@ class DAGScheduler(
callSite: Option[String] = None)
: Stage =
{
- if (shuffleDep != None) {
- // Kind of ugly: need to register RDDs with the cache and map output tracker here
- // since we can't do it in the RDD constructor because # of partitions is unknown
- logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")")
- mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size)
- }
val id = nextStageId.getAndIncrement()
val stage =
new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite)
stageIdToStage(id) = stage
+ updateJobIdStageIdMaps(jobId, stage)
stageToInfos(stage) = new StageInfo(stage)
stage
}
/**
+ * Create a shuffle map Stage for the given RDD. The stage will also be associated with the
+ * provided jobId. If a stage for the shuffleId existed previously so that the shuffleId is
+ * present in the MapOutputTracker, then the number and location of available outputs are
+ * recovered from the MapOutputTracker
+ */
+ private def newOrUsedStage(
+ rdd: RDD[_],
+ numTasks: Int,
+ shuffleDep: ShuffleDependency[_,_],
+ jobId: Int,
+ callSite: Option[String] = None)
+ : Stage =
+ {
+ val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite)
+ if (mapOutputTracker.has(shuffleDep.shuffleId)) {
+ val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId)
+ val locs = MapOutputTracker.deserializeMapStatuses(serLocs)
+ for (i <- 0 until locs.size) stage.outputLocs(i) = List(locs(i))
+ stage.numAvailableOutputs = locs.size
+ } else {
+ // Kind of ugly: need to register RDDs with the cache and map output tracker here
+ // since we can't do it in the RDD constructor because # of partitions is unknown
+ logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")")
+ mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.size)
+ }
+ stage
+ }
+
+ /**
* Get or create the list of parent stages for a given RDD. The stages will be assigned the
* provided jobId if they haven't already been created with a lower jobId.
*/
@@ -287,6 +352,94 @@ class DAGScheduler(
}
/**
+ * Registers the given jobId among the jobs that need the given stage and
+ * all of that stage's ancestors.
+ */
+ private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) {
+ def updateJobIdStageIdMapsList(stages: List[Stage]) {
+ if (!stages.isEmpty) {
+ val s = stages.head
+ stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId
+ jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id
+ val parents = getParentStages(s.rdd, jobId)
+ val parentsWithoutThisJobId = parents.filter(p =>
+ !stageIdToJobIds.get(p.id).exists(_.contains(jobId)))
+ updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail)
+ }
+ }
+ updateJobIdStageIdMapsList(List(stage))
+ }
+
+ /**
+ * Removes job and any stages that are not needed by any other job. Returns the set of ids for
+ * stages that were removed. The associated tasks for those stages need to be cancelled if we
+ * got here via job cancellation.
+ */
+ private def removeJobAndIndependentStages(jobId: Int): Set[Int] = {
+ val registeredStages = jobIdToStageIds(jobId)
+ val independentStages = new HashSet[Int]()
+ if (registeredStages.isEmpty) {
+ logError("No stages registered for job " + jobId)
+ } else {
+ stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach {
+ case (stageId, jobSet) =>
+ if (!jobSet.contains(jobId)) {
+ logError(
+ "Job %d not registered for stage %d even though that stage was registered for the job"
+ .format(jobId, stageId))
+ } else {
+ def removeStage(stageId: Int) {
+ // data structures based on Stage
+ stageIdToStage.get(stageId).foreach { s =>
+ if (running.contains(s)) {
+ logDebug("Removing running stage %d".format(stageId))
+ running -= s
+ }
+ stageToInfos -= s
+ shuffleToMapStage.keys.filter(shuffleToMapStage(_) == s).foreach(shuffleId =>
+ shuffleToMapStage.remove(shuffleId))
+ if (pendingTasks.contains(s) && !pendingTasks(s).isEmpty) {
+ logDebug("Removing pending status for stage %d".format(stageId))
+ }
+ pendingTasks -= s
+ if (waiting.contains(s)) {
+ logDebug("Removing stage %d from waiting set.".format(stageId))
+ waiting -= s
+ }
+ if (failed.contains(s)) {
+ logDebug("Removing stage %d from failed set.".format(stageId))
+ failed -= s
+ }
+ }
+ // data structures based on StageId
+ stageIdToStage -= stageId
+ stageIdToJobIds -= stageId
+
+ logDebug("After removal of stage %d, remaining stages = %d"
+ .format(stageId, stageIdToStage.size))
+ }
+
+ jobSet -= jobId
+ if (jobSet.isEmpty) { // no other job needs this stage
+ independentStages += stageId
+ removeStage(stageId)
+ }
+ }
+ }
+ }
+ independentStages.toSet
+ }
+
+ private def jobIdToStageIdsRemove(jobId: Int) {
+ if (!jobIdToStageIds.contains(jobId)) {
+ logDebug("Trying to remove unregistered job " + jobId)
+ } else {
+ removeJobAndIndependentStages(jobId)
+ jobIdToStageIds -= jobId
+ }
+ }
+
+ /**
* 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.
*/
@@ -315,11 +468,12 @@ class DAGScheduler(
assert(partitions.size > 0)
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
- eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)
+ eventProcessActor ! JobSubmitted(
+ jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)
waiter
}
- def runJob[T, U: ClassManifest](
+ def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
@@ -350,7 +504,8 @@ class DAGScheduler(
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
val partitions = (0 until rdd.partitions.size).toArray
val jobId = nextJobId.getAndIncrement()
- eventProcessActor ! JobSubmitted(jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties)
+ eventProcessActor ! JobSubmitted(
+ jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties)
listener.awaitResult() // Will throw an exception if the job fails
}
@@ -375,13 +530,25 @@ class DAGScheduler(
}
/**
- * Process one event retrieved from the event queue.
- * Returns true if we should stop the event loop.
+ * Process one event retrieved from the event processing actor.
+ *
+ * @param event The event to be processed.
+ * @return `true` if we should stop the event loop.
*/
private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = {
event match {
case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
- val finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite))
+ var finalStage: Stage = null
+ try {
+ // New stage creation may throw an exception if, for example, jobs are run on a HadoopRDD
+ // whose underlying HDFS files have been deleted.
+ finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite))
+ } catch {
+ case e: Exception =>
+ logWarning("Creating new stage failed due to exception - job: " + jobId, e)
+ listener.jobFailed(e)
+ return false
+ }
val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties)
clearCacheLocs()
logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length +
@@ -391,37 +558,32 @@ class DAGScheduler(
logInfo("Missing parents: " + getMissingParentStages(finalStage))
if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) {
// Compute very short actions like first() or take() with no parent stages locally.
+ listenerBus.post(SparkListenerJobStart(job, Array(), properties))
runLocally(job)
} else {
- listenerBus.post(SparkListenerJobStart(job, properties))
idToActiveJob(jobId) = job
activeJobs += job
resultStageToJob(finalStage) = job
+ listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties))
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)
- }
+ handleJobCancellation(jobId)
case JobGroupCancelled(groupId) =>
// Cancel all jobs belonging to this job group.
// First finds all active jobs with this group id, and then kill stages for them.
- val jobIds = activeJobs.filter(groupId == _.properties.get(SparkContext.SPARK_JOB_GROUP_ID))
- .map(_.jobId)
- if (!jobIds.isEmpty) {
- running.filter(stage => jobIds.contains(stage.jobId)).foreach { stage =>
- taskSched.cancelTasks(stage.id)
- }
- }
+ val activeInGroup = activeJobs.filter(activeJob =>
+ groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID))
+ val jobIds = activeInGroup.map(_.jobId)
+ jobIds.foreach { handleJobCancellation }
case AllJobsCancelled =>
// Cancel all running jobs.
- running.foreach { stage =>
- taskSched.cancelTasks(stage.id)
- }
+ running.map(_.jobId).foreach { handleJobCancellation }
+ activeJobs.clear() // These should already be empty by this point,
+ idToActiveJob.clear() // but just in case we lost track of some jobs...
case ExecutorGained(execId, host) =>
handleExecutorGained(execId, host)
@@ -430,6 +592,19 @@ class DAGScheduler(
handleExecutorLost(execId)
case BeginEvent(task, taskInfo) =>
+ for (
+ job <- idToActiveJob.get(task.stageId);
+ stage <- stageIdToStage.get(task.stageId);
+ stageInfo <- stageToInfos.get(stage)
+ ) {
+ if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 &&
+ !stageInfo.emittedTaskSizeWarning) {
+ stageInfo.emittedTaskSizeWarning = true
+ logWarning(("Stage %d (%s) contains a task of very large " +
+ "size (%d KB). The maximum recommended task size is %d KB.").format(
+ task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN))
+ }
+ }
listenerBus.post(SparkListenerTaskStart(task, taskInfo))
case GettingResultEvent(task, taskInfo) =>
@@ -440,7 +615,12 @@ class DAGScheduler(
handleTaskCompletion(completion)
case TaskSetFailed(taskSet, reason) =>
- abortStage(stageIdToStage(taskSet.stageId), reason)
+ stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) }
+
+ case ResubmitFailedStages =>
+ if (failed.size > 0) {
+ resubmitFailedStages()
+ }
case StopDAGScheduler =>
// Cancel any active jobs
@@ -502,6 +682,7 @@ class DAGScheduler(
// Broken out for easier testing in DAGSchedulerSuite.
protected def runLocallyWithinThread(job: ActiveJob) {
+ var jobResult: JobResult = JobSucceeded
try {
SparkEnv.set(env)
val rdd = job.finalStage.rdd
@@ -516,31 +697,59 @@ class DAGScheduler(
}
} catch {
case e: Exception =>
+ jobResult = JobFailed(e, Some(job.finalStage))
job.listener.jobFailed(e)
+ } finally {
+ val s = job.finalStage
+ stageIdToJobIds -= s.id // clean up data structures that were populated for a local job,
+ stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through
+ stageToInfos -= s // completion events or stage abort
+ jobIdToStageIds -= job.jobId
+ listenerBus.post(SparkListenerJobEnd(job, jobResult))
+ }
+ }
+
+ /** Finds the earliest-created active job that needs the stage */
+ // TODO: Probably should actually find among the active jobs that need this
+ // stage the one with the highest priority (highest-priority pool, earliest created).
+ // That should take care of at least part of the priority inversion problem with
+ // cross-job dependencies.
+ private def activeJobForStage(stage: Stage): Option[Int] = {
+ if (stageIdToJobIds.contains(stage.id)) {
+ val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted
+ jobsThatUseStage.find(idToActiveJob.contains(_))
+ } else {
+ None
}
}
/** Submits stage, but first recursively submits any missing parents. */
private def submitStage(stage: Stage) {
- logDebug("submitStage(" + stage + ")")
- if (!waiting(stage) && !running(stage) && !failed(stage)) {
- val missing = getMissingParentStages(stage).sortBy(_.id)
- logDebug("missing: " + missing)
- if (missing == Nil) {
- logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents")
- submitMissingTasks(stage)
- running += stage
- } else {
- for (parent <- missing) {
- submitStage(parent)
+ val jobId = activeJobForStage(stage)
+ if (jobId.isDefined) {
+ logDebug("submitStage(" + stage + ")")
+ if (!waiting(stage) && !running(stage) && !failed(stage)) {
+ val missing = getMissingParentStages(stage).sortBy(_.id)
+ logDebug("missing: " + missing)
+ if (missing == Nil) {
+ logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents")
+ submitMissingTasks(stage, jobId.get)
+ running += stage
+ } else {
+ for (parent <- missing) {
+ submitStage(parent)
+ }
+ waiting += stage
}
- waiting += stage
}
+ } else {
+ abortStage(stage, "No active job for stage " + stage.id)
}
}
+
/** Called when stage's parents are available and we can now do its task. */
- private def submitMissingTasks(stage: Stage) {
+ private def submitMissingTasks(stage: Stage, jobId: Int) {
logDebug("submitMissingTasks(" + stage + ")")
// Get our pending tasks and remember them in our pendingTasks entry
val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet)
@@ -561,7 +770,7 @@ class DAGScheduler(
}
}
- val properties = if (idToActiveJob.contains(stage.jobId)) {
+ val properties = if (idToActiveJob.contains(jobId)) {
idToActiveJob(stage.jobId).properties
} else {
//this stage will be assigned to "default" pool
@@ -619,7 +828,7 @@ class DAGScheduler(
}
logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime))
stageToInfos(stage).completionTime = Some(System.currentTimeMillis())
- listenerBus.post(StageCompleted(stageToInfos(stage)))
+ listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage)))
running -= stage
}
event.reason match {
@@ -643,6 +852,7 @@ class DAGScheduler(
activeJobs -= job
resultStageToJob -= stage
markStageAsFinished(stage)
+ jobIdToStageIdsRemove(job.jobId)
listenerBus.post(SparkListenerJobEnd(job, JobSucceeded))
}
job.listener.taskSucceeded(rt.outputId, event.result)
@@ -679,7 +889,7 @@ class DAGScheduler(
changeEpoch = true)
}
clearCacheLocs()
- if (stage.outputLocs.count(_ == Nil) != 0) {
+ if (stage.outputLocs.exists(_ == Nil)) {
// Some tasks had failed; let's resubmit this stage
// TODO: Lower-level scheduler should also deal with this
logInfo("Resubmitting " + stage + " (" + stage.name +
@@ -696,9 +906,12 @@ class DAGScheduler(
}
waiting --= newlyRunnable
running ++= newlyRunnable
- for (stage <- newlyRunnable.sortBy(_.id)) {
+ for {
+ stage <- newlyRunnable.sortBy(_.id)
+ jobId <- activeJobForStage(stage)
+ } {
logInfo("Submitting " + stage + " (" + stage.rdd + "), which is now runnable")
- submitMissingTasks(stage)
+ submitMissingTasks(stage, jobId)
}
}
}
@@ -782,21 +995,42 @@ class DAGScheduler(
}
}
+ private def handleJobCancellation(jobId: Int) {
+ if (!jobIdToStageIds.contains(jobId)) {
+ logDebug("Trying to cancel unregistered job " + jobId)
+ } else {
+ val independentStages = removeJobAndIndependentStages(jobId)
+ independentStages.foreach { taskSched.cancelTasks }
+ val error = new SparkException("Job %d cancelled".format(jobId))
+ val job = idToActiveJob(jobId)
+ job.listener.jobFailed(error)
+ jobIdToStageIds -= jobId
+ activeJobs -= job
+ idToActiveJob -= jobId
+ listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage))))
+ }
+ }
+
/**
* Aborts all jobs depending on a particular Stage. This is called in response to a task set
* being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside.
*/
private def abortStage(failedStage: Stage, reason: String) {
+ if (!stageIdToStage.contains(failedStage.id)) {
+ // Skip all the actions if the stage has been removed.
+ return
+ }
val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq
stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis())
for (resultStage <- dependentStages) {
val job = resultStageToJob(resultStage)
val error = new SparkException("Job aborted: " + reason)
job.listener.jobFailed(error)
- listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))
+ jobIdToStageIdsRemove(job.jobId)
idToActiveJob -= resultStage.jobId
activeJobs -= job
resultStageToJob -= resultStage
+ listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))
}
if (dependentStages.isEmpty) {
logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done")
@@ -867,25 +1101,24 @@ class DAGScheduler(
}
private def cleanup(cleanupTime: Long) {
- var sizeBefore = stageIdToStage.size
- stageIdToStage.clearOldValues(cleanupTime)
- logInfo("stageIdToStage " + sizeBefore + " --> " + stageIdToStage.size)
-
- sizeBefore = shuffleToMapStage.size
- shuffleToMapStage.clearOldValues(cleanupTime)
- logInfo("shuffleToMapStage " + sizeBefore + " --> " + shuffleToMapStage.size)
-
- sizeBefore = pendingTasks.size
- pendingTasks.clearOldValues(cleanupTime)
- logInfo("pendingTasks " + sizeBefore + " --> " + pendingTasks.size)
-
- sizeBefore = stageToInfos.size
- stageToInfos.clearOldValues(cleanupTime)
- logInfo("stageToInfos " + sizeBefore + " --> " + stageToInfos.size)
+ Map(
+ "stageIdToStage" -> stageIdToStage,
+ "shuffleToMapStage" -> shuffleToMapStage,
+ "pendingTasks" -> pendingTasks,
+ "stageToInfos" -> stageToInfos,
+ "jobIdToStageIds" -> jobIdToStageIds,
+ "stageIdToJobIds" -> stageIdToJobIds).
+ foreach { case(s, t) => {
+ val sizeBefore = t.size
+ t.clearOldValues(cleanupTime)
+ logInfo("%s %d --> %d".format(s, sizeBefore, t.size))
+ }}
}
def stop() {
- eventProcessActor ! StopDAGScheduler
+ if (eventProcessActor != null) {
+ eventProcessActor ! StopDAGScheduler
+ }
metadataCleaner.cancel()
taskSched.stop()
}
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 708d221d60..add1187613 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
@@ -65,12 +65,13 @@ private[scheduler] case class CompletionEvent(
taskMetrics: TaskMetrics)
extends DAGSchedulerEvent
-private[scheduler]
-case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent
+private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent
private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent
private[scheduler]
case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent
+private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent
+
private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala
index 5077b2b48b..2bc43a9186 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorLossReason.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorLossReason.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import org.apache.spark.executor.ExecutorExitCode
diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 1791ee660d..90eb8a747f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
@@ -32,7 +32,7 @@ import scala.collection.JavaConversions._
/**
* Parses and holds information about inputFormat (and files) specified as a parameter.
*/
-class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_],
+class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_],
val path: String) extends Logging {
var mapreduceInputFormat: Boolean = false
@@ -40,7 +40,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
validate()
- override def toString(): String = {
+ override def toString: String = {
"InputFormatInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", path : " + path
}
@@ -125,7 +125,7 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
}
private def findPreferredLocations(): Set[SplitInfo] = {
- logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat +
+ logDebug("mapreduceInputFormat : " + mapreduceInputFormat + ", mapredInputFormat : " + mapredInputFormat +
", inputFormatClazz : " + inputFormatClazz)
if (mapreduceInputFormat) {
return prefLocsFromMapreduceInputFormat()
@@ -143,14 +143,14 @@ class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Cl
object InputFormatInfo {
/**
Computes the preferred locations based on input(s) and returned a location to block map.
- Typical use of this method for allocation would follow some algo like this
- (which is what we currently do in YARN branch) :
+ Typical use of this method for allocation would follow some algo like this:
+
a) For each host, count number of splits hosted on that host.
b) Decrement the currently allocated containers on that host.
c) Compute rack info for each host and update rack -> count map based on (b).
d) Allocate nodes based on (c)
- e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node
- (even if data locality on that is very high) : this is to prevent fragility of job if a single
+ e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node
+ (even if data locality on that is very high) : this is to prevent fragility of job if a single
(or small set of) hosts go down.
go to (a) until required nodes are allocated.
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 60927831a1..f8fa5a9f7a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala
@@ -297,7 +297,7 @@ class JobLogger(val user: String, val logDirName: String)
* When stage is completed, record stage completion status
* @param stageCompleted Stage completed event
*/
- override def onStageCompleted(stageCompleted: StageCompleted) {
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) {
stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format(
stageCompleted.stage.stageId))
}
@@ -328,10 +328,6 @@ class JobLogger(val user: String, val logDirName: String)
task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" +
mapId + " REDUCE_ID=" + reduceId
stageLogInfo(task.stageId, taskStatus)
- case OtherFailure(message) =>
- taskStatus += " STATUS=FAILURE TID=" + taskInfo.taskId +
- " STAGE_ID=" + task.stageId + " INFO=" + message
- stageLogInfo(task.stageId, taskStatus)
case _ =>
}
}
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 58f238d8cf..b026f860a8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala
@@ -31,6 +31,7 @@ private[spark] class JobWaiter[T](
private var finishedTasks = 0
// Is the job as a whole finished (succeeded or failed)?
+ @volatile
private var _jobFinished = totalTasks == 0
def jobFinished = _jobFinished
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
index 596f9adde9..1791242215 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala
@@ -117,8 +117,4 @@ private[spark] class Pool(
parent.decreaseRunningTasks(taskNum)
}
}
-
- override def hasPendingTasks(): Boolean = {
- schedulableQueue.exists(_.hasPendingTasks())
- }
}
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 310ec62ca8..28f3ba53b8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -32,7 +32,9 @@ 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(MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues)
+ // TODO: This object shouldn't have global variables
+ val metadataCleaner = new MetadataCleaner(
+ MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf)
def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = {
synchronized {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
index 1c7ea2dccc..d573e125a3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Schedulable.scala
@@ -42,5 +42,4 @@ private[spark] trait Schedulable {
def executorLost(executorId: String, host: String): Unit
def checkSpeculatableTasks(): Boolean
def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager]
- def hasPendingTasks(): Boolean
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
index 356fe56bf3..3cf995ea74 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala
@@ -20,7 +20,7 @@ package org.apache.spark.scheduler
import java.io.{FileInputStream, InputStream}
import java.util.{NoSuchElementException, Properties}
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
import scala.xml.XML
@@ -49,10 +49,10 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool)
}
}
-private[spark] class FairSchedulableBuilder(val rootPool: Pool)
+private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf)
extends SchedulableBuilder with Logging {
- val schedulerAllocFile = Option(System.getProperty("spark.scheduler.allocation.file"))
+ val schedulerAllocFile = conf.getOption("spark.scheduler.allocation.file")
val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml"
val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool"
val DEFAULT_POOL_NAME = "default"
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
index 5367218faa..02bdbba825 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
@@ -15,12 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import org.apache.spark.SparkContext
/**
- * A backend interface for cluster scheduling systems that allows plugging in different ones under
+ * A backend interface for scheduling systems that allows plugging in different ones under
* ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as
* machines become available and can launch tasks on them.
*/
@@ -31,7 +31,4 @@ private[spark] trait SchedulerBackend {
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
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
index 0a786deb16..3832ee7ff6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulingMode.scala
@@ -22,7 +22,7 @@ package org.apache.spark.scheduler
* to order tasks amongst a Schedulable's sub-queues
* "NONE" is used when the a Schedulable has no sub-queues.
*/
-object SchedulingMode extends Enumeration("FAIR", "FIFO", "NONE") {
+object SchedulingMode extends Enumeration {
type SchedulingMode = Value
val FAIR,FIFO,NONE = Value
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 0f2deb4bcb..a37ead5632 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -37,7 +37,9 @@ 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(MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues)
+ // TODO: This object shouldn't have global variables
+ val metadataCleaner = new MetadataCleaner(
+ MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf)
def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
synchronized {
@@ -152,7 +154,7 @@ private[spark] class ShuffleMapTask(
try {
// Obtain all the block writers for shuffle blocks.
- val ser = SparkEnv.get.serializerManager.get(dep.serializerClass)
+ val ser = SparkEnv.get.serializerManager.get(dep.serializerClass, SparkEnv.get.conf)
shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser)
// Write the map output to its associated buckets.
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 a35081f7b1..627995c826 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -27,7 +27,7 @@ sealed trait SparkListenerEvents
case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties)
extends SparkListenerEvents
-case class StageCompleted(val stage: StageInfo) extends SparkListenerEvents
+case class SparkListenerStageCompleted(val stage: StageInfo) extends SparkListenerEvents
case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents
@@ -37,7 +37,7 @@ case class SparkListenerTaskGettingResult(
case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo,
taskMetrics: TaskMetrics) extends SparkListenerEvents
-case class SparkListenerJobStart(job: ActiveJob, properties: Properties = null)
+case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], properties: Properties = null)
extends SparkListenerEvents
case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult)
@@ -47,7 +47,7 @@ trait SparkListener {
/**
* Called when a stage is completed, with information on the completed stage
*/
- def onStageCompleted(stageCompleted: StageCompleted) { }
+ def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { }
/**
* Called when a stage is submitted
@@ -63,7 +63,7 @@ trait SparkListener {
* Called when a task begins remotely fetching its result (will not be called for tasks that do
* not need to fetch the result remotely).
*/
- def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { }
+ def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { }
/**
* Called when a task ends
@@ -86,7 +86,7 @@ trait SparkListener {
* Simple SparkListener that logs a few summary statistics when each stage completes
*/
class StatsReportListener extends SparkListener with Logging {
- override def onStageCompleted(stageCompleted: StageCompleted) {
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) {
import org.apache.spark.scheduler.StatsReportListener._
implicit val sc = stageCompleted
this.logInfo("Finished stage: " + stageCompleted.stage)
@@ -119,20 +119,24 @@ object StatsReportListener extends Logging {
val probabilities = percentiles.map{_ / 100.0}
val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%"
- def extractDoubleDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Double]): Option[Distribution] = {
+ def extractDoubleDistribution(stage: SparkListenerStageCompleted,
+ getMetric: (TaskInfo,TaskMetrics) => Option[Double])
+ : Option[Distribution] = {
Distribution(stage.stage.taskInfos.flatMap {
case ((info,metric)) => getMetric(info, metric)})
}
//is there some way to setup the types that I can get rid of this completely?
- def extractLongDistribution(stage:StageCompleted, getMetric: (TaskInfo,TaskMetrics) => Option[Long]): Option[Distribution] = {
+ def extractLongDistribution(stage: SparkListenerStageCompleted,
+ getMetric: (TaskInfo,TaskMetrics) => Option[Long])
+ : Option[Distribution] = {
extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble})
}
def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) {
val stats = d.statCounter
- logInfo(heading + stats)
val quantiles = d.getQuantiles(probabilities).map{formatNumber}
+ logInfo(heading + stats)
logInfo(percentilesHeader)
logInfo("\t" + quantiles.mkString("\t"))
}
@@ -147,12 +151,12 @@ object StatsReportListener extends Logging {
}
def showDistribution(heading:String, format: String, getMetric: (TaskInfo,TaskMetrics) => Option[Double])
- (implicit stage: StageCompleted) {
+ (implicit stage: SparkListenerStageCompleted) {
showDistribution(heading, extractDoubleDistribution(stage, getMetric), format)
}
def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long])
- (implicit stage: StageCompleted) {
+ (implicit stage: SparkListenerStageCompleted) {
showBytesDistribution(heading, extractLongDistribution(stage, getMetric))
}
@@ -169,12 +173,10 @@ object StatsReportListener extends Logging {
}
def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long])
- (implicit stage: StageCompleted) {
+ (implicit stage: SparkListenerStageCompleted) {
showMillisDistribution(heading, extractLongDistribution(stage, getMetric))
}
-
-
val seconds = 1000L
val minutes = seconds * 60
val hours = minutes * 60
@@ -198,7 +200,6 @@ object StatsReportListener extends Logging {
}
-
case class RuntimePercentage(executorPct: Double, fetchPct: Option[Double], other: Double)
object RuntimePercentage {
def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = {
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 d5824e7954..e7defd768b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -41,7 +41,7 @@ private[spark] class SparkListenerBus() extends Logging {
event match {
case stageSubmitted: SparkListenerStageSubmitted =>
sparkListeners.foreach(_.onStageSubmitted(stageSubmitted))
- case stageCompleted: StageCompleted =>
+ case stageCompleted: SparkListenerStageCompleted =>
sparkListeners.foreach(_.onStageCompleted(stageCompleted))
case jobStart: SparkListenerJobStart =>
sparkListeners.foreach(_.onJobStart(jobStart))
@@ -91,4 +91,3 @@ private[spark] class SparkListenerBus() extends Logging {
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 93599dfdc8..e9f2198a00 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -33,4 +33,5 @@ class StageInfo(
val name = stage.name
val numPartitions = stage.numPartitions
val numTasks = stage.numTasks
+ var emittedTaskSizeWarning = false
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
index 4bae26f3a6..3c22edd524 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala
@@ -46,6 +46,8 @@ class TaskInfo(
var failed = false
+ var serializedSize: Int = 0
+
def markGettingResult(time: Long = System.currentTimeMillis) {
gettingResultTime = time
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
index 47b0f387aa..35de13c385 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala
@@ -18,9 +18,7 @@
package org.apache.spark.scheduler
-private[spark] object TaskLocality
- extends Enumeration("PROCESS_LOCAL", "NODE_LOCAL", "RACK_LOCAL", "ANY")
-{
+private[spark] object TaskLocality extends Enumeration {
// process local is expected to be used ONLY within tasksetmanager for now.
val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
index 7e468d0d67..e80cc6b0f6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala
@@ -35,18 +35,15 @@ case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Se
/** A TaskResult that contains the task's return value and accumulator updates. */
private[spark]
-class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics)
+class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long, Any], var metrics: TaskMetrics)
extends TaskResult[T] with Externalizable {
- def this() = this(null.asInstanceOf[T], null, null)
+ def this() = this(null.asInstanceOf[ByteBuffer], null, null)
override def writeExternal(out: ObjectOutput) {
- val objectSer = SparkEnv.get.serializer.newInstance()
- val bb = objectSer.serialize(value)
-
- out.writeInt(bb.remaining())
- Utils.writeByteBuffer(bb, out)
+ out.writeInt(valueBytes.remaining);
+ Utils.writeByteBuffer(valueBytes, out)
out.writeInt(accumUpdates.size)
for ((key, value) <- accumUpdates) {
@@ -58,12 +55,10 @@ class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var me
override def readExternal(in: ObjectInput) {
- val objectSer = SparkEnv.get.serializer.newInstance()
-
val blen = in.readInt()
val byteVal = new Array[Byte](blen)
in.readFully(byteVal)
- value = objectSer.deserialize(ByteBuffer.wrap(byteVal))
+ valueBytes = ByteBuffer.wrap(byteVal)
val numUpdates = in.readInt
if (numUpdates == 0) {
@@ -76,4 +71,9 @@ class DirectTaskResult[T](var value: T, var accumUpdates: Map[Long, Any], var me
}
metrics = in.readObject().asInstanceOf[TaskMetrics]
}
+
+ def value(): T = {
+ val resultSer = SparkEnv.get.serializer.newInstance()
+ return resultSer.deserialize(valueBytes)
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
index 2064d97b49..e22b1e53e8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/TaskResultGetter.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala
@@ -15,23 +15,23 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
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
import org.apache.spark.util.Utils
/**
* Runs a thread pool that deserializes and remotely fetches (if necessary) task results.
*/
-private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterScheduler)
+private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl)
extends Logging {
- private val THREADS = System.getProperty("spark.resultGetter.threads", "4").toInt
+
+ private val THREADS = sparkEnv.conf.get("spark.resultGetter.threads", "4").toInt
private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool(
THREADS, "Result resolver thread")
@@ -42,7 +42,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche
}
def enqueueSuccessfulTask(
- taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) {
+ taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) {
getTaskResultExecutor.execute(new Runnable {
override def run() {
try {
@@ -71,14 +71,14 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche
case cnf: ClassNotFoundException =>
val loader = Thread.currentThread.getContextClassLoader
taskSetManager.abort("ClassNotFound with classloader: " + loader)
- case ex =>
+ case ex: Throwable =>
taskSetManager.abort("Exception while deserializing and fetching task: %s".format(ex))
}
}
})
}
- def enqueueFailedTask(taskSetManager: ClusterTaskSetManager, tid: Long, taskState: TaskState,
+ def enqueueFailedTask(taskSetManager: TaskSetManager, tid: Long, taskState: TaskState,
serializedData: ByteBuffer) {
var reason: Option[TaskEndReason] = None
getTaskResultExecutor.execute(new Runnable {
@@ -95,7 +95,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSche
val loader = Thread.currentThread.getContextClassLoader
logError(
"Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader)
- case ex => {}
+ case ex: Throwable => {}
}
scheduler.handleFailedTask(taskSetManager, tid, taskState, reason)
}
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 10e0478108..17b6d97e90 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -20,11 +20,12 @@ package org.apache.spark.scheduler
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.
+ * Low-level task scheduler interface, currently implemented exclusively by the ClusterScheduler.
+ * This interface allows plugging in different task schedulers. Each TaskScheduler schedulers tasks
+ * 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.
*/
private[spark] trait TaskScheduler {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index c1e65a3c48..0c8ed62759 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.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 java.util.concurrent.atomic.AtomicLong
@@ -24,41 +24,46 @@ import java.util.{TimerTask, Timer}
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
import scala.collection.mutable.HashSet
-
-import akka.util.duration._
+import scala.concurrent.duration._
import org.apache.spark._
import org.apache.spark.TaskState.TaskState
-import org.apache.spark.scheduler._
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
/**
- * The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call
- * initialize() and start(), then submit task sets through the runTasks method.
- *
- * This class can work with multiple types of clusters by acting through a SchedulerBackend.
+ * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend.
+ * It can also work with a local setup by using a LocalBackend and setting isLocal to true.
* It handles common logic, like determining a scheduling order across jobs, waking up to launch
* speculative tasks, etc.
*
+ * Clients should first call initialize() and start(), then submit task sets through the
+ * runTasks method.
+ *
* THREADING: SchedulerBackends and task-submitting clients can call this class from multiple
* threads, so it needs locks in public API methods to maintain its state. In addition, some
* SchedulerBackends sycnchronize on themselves when they want to send events here, and then
* acquire a lock on us, so we need to make sure that we don't try to lock the backend while
* we are holding a lock on ourselves.
*/
-private[spark] class ClusterScheduler(val sc: SparkContext)
- extends TaskScheduler
- with Logging
+private[spark] class TaskSchedulerImpl(
+ val sc: SparkContext,
+ val maxTaskFailures: Int,
+ isLocal: Boolean = false)
+ extends TaskScheduler with Logging
{
+ def this(sc: SparkContext) = this(sc, sc.conf.get("spark.task.maxFailures", "4").toInt)
+
+ val conf = sc.conf
+
// How often to check for speculative tasks
- val SPECULATION_INTERVAL = System.getProperty("spark.speculation.interval", "100").toLong
+ val SPECULATION_INTERVAL = conf.get("spark.speculation.interval", "100").toLong
// Threshold above which we warn user initial TaskSet may be starved
- val STARVATION_TIMEOUT = System.getProperty("spark.starvation.timeout", "15000").toLong
+ val STARVATION_TIMEOUT = conf.get("spark.starvation.timeout", "15000").toLong
- // ClusterTaskSetManagers are not thread safe, so any access to one should be synchronized
+ // TaskSetManagers are not thread safe, so any access to one should be synchronized
// on this class.
- val activeTaskSets = new HashMap[String, ClusterTaskSetManager]
+ val activeTaskSets = new HashMap[String, TaskSetManager]
val taskIdToTaskSetId = new HashMap[Long, String]
val taskIdToExecutorId = new HashMap[Long, String]
@@ -91,7 +96,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
var rootPool: Pool = null
// default scheduler is FIFO
val schedulingMode: SchedulingMode = SchedulingMode.withName(
- System.getProperty("spark.scheduler.mode", "FIFO"))
+ conf.get("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)
@@ -100,8 +105,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
this.dagScheduler = dagScheduler
}
- def initialize(context: SchedulerBackend) {
- backend = context
+ def initialize(backend: SchedulerBackend) {
+ this.backend = backend
// temporarily set rootPool name to empty
rootPool = new Pool("", schedulingMode, 0, 0)
schedulableBuilder = {
@@ -109,7 +114,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
case SchedulingMode.FIFO =>
new FIFOSchedulableBuilder(rootPool)
case SchedulingMode.FAIR =>
- new FairSchedulableBuilder(rootPool)
+ new FairSchedulableBuilder(rootPool, conf)
}
}
schedulableBuilder.buildPools()
@@ -120,9 +125,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
override def start() {
backend.start()
- if (System.getProperty("spark.speculation", "false").toBoolean) {
+ if (!isLocal && conf.get("spark.speculation", "false").toBoolean) {
logInfo("Starting speculative execution thread")
-
+ import sc.env.actorSystem.dispatcher
sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds,
SPECULATION_INTERVAL milliseconds) {
checkSpeculatableTasks()
@@ -134,12 +139,12 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
val tasks = taskSet.tasks
logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks")
this.synchronized {
- val manager = new ClusterTaskSetManager(this, taskSet)
+ val manager = new TaskSetManager(this, taskSet, maxTaskFailures)
activeTaskSets(taskSet.id) = manager
schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties)
taskSetTaskIds(taskSet.id) = new HashSet[Long]()
- if (!hasReceivedTask) {
+ if (!isLocal && !hasReceivedTask) {
starvationTimer.scheduleAtFixedRate(new TimerTask() {
override def run() {
if (!hasLaunchedTask) {
@@ -173,7 +178,9 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
backend.killTask(tid, execId)
}
}
- tsm.error("Stage %d was cancelled".format(stageId))
+ logInfo("Stage %d was cancelled".format(stageId))
+ tsm.removeAllRunningTasks()
+ taskSetFinished(tsm)
}
}
@@ -278,7 +285,8 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
}
case None =>
- logInfo("Ignoring update from TID " + tid + " because its task set is gone")
+ logInfo("Ignoring update with state %s from TID %s because its task set is gone"
+ .format(state, tid))
}
} catch {
case e: Exception => logError("Exception in statusUpdate", e)
@@ -291,19 +299,19 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
}
- def handleTaskGettingResult(taskSetManager: ClusterTaskSetManager, tid: Long) {
+ def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) {
taskSetManager.handleTaskGettingResult(tid)
}
def handleSuccessfulTask(
- taskSetManager: ClusterTaskSetManager,
+ taskSetManager: TaskSetManager,
tid: Long,
taskResult: DirectTaskResult[_]) = synchronized {
taskSetManager.handleSuccessfulTask(tid, taskResult)
}
def handleFailedTask(
- taskSetManager: ClusterTaskSetManager,
+ taskSetManager: TaskSetManager,
tid: Long,
taskState: TaskState,
reason: Option[TaskEndReason]) = synchronized {
@@ -321,7 +329,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
// Have each task set throw a SparkException with the error
for ((taskSetId, manager) <- activeTaskSets) {
try {
- manager.error(message)
+ manager.abort(message)
} catch {
case e: Exception => logError("Exception in error callback", e)
}
@@ -351,7 +359,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
override def defaultParallelism() = backend.defaultParallelism()
-
// Check for speculatable tasks in all our active jobs.
def checkSpeculatableTasks() {
var shouldRevive = false
@@ -363,13 +370,6 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
}
- // Check for pending tasks in all our active jobs.
- def hasPendingTasks: Boolean = {
- synchronized {
- rootPool.hasPendingTasks()
- }
- }
-
def executorLost(executorId: String, reason: ExecutorLossReason) {
var failedExecutor: Option[String] = None
@@ -428,7 +428,7 @@ private[spark] class ClusterScheduler(val sc: SparkContext)
}
-object ClusterScheduler {
+private[spark] object TaskSchedulerImpl {
/**
* Used to balance containers across hosts.
*
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 90f6bcefac..6dd1469d8f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -17,32 +17,693 @@
package org.apache.spark.scheduler
-import java.nio.ByteBuffer
+import java.io.NotSerializableException
+import java.util.Arrays
+import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.HashSet
+import scala.math.max
+import scala.math.min
+
+import org.apache.spark.{ExceptionFailure, FetchFailed, Logging, Resubmitted, SparkEnv,
+ Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState}
import org.apache.spark.TaskState.TaskState
+import org.apache.spark.util.{Clock, SystemClock}
+
/**
- * Tracks and schedules the tasks within a single TaskSet. This class keeps track of the status of
- * each task and is responsible for retries on failure and locality. The main interfaces to it
- * are resourceOffer, which asks the TaskSet whether it wants to run a task on one node, and
- * statusUpdate, which tells it that one of its tasks changed state (e.g. finished).
+ * Schedules the tasks within a single TaskSet in the ClusterScheduler. This class keeps track of
+ * each task, retries tasks if they fail (up to a limited number of times), and
+ * handles locality-aware scheduling for this TaskSet via delay scheduling. The main interfaces
+ * to it are resourceOffer, which asks the TaskSet whether it wants to run a task on one node,
+ * and statusUpdate, which tells it that one of its tasks changed state (e.g. finished).
+ *
+ * THREADING: This class is designed to only be called from code with a lock on the
+ * TaskScheduler (e.g. its event handlers). It should not be called from other threads.
*
- * THREADING: This class is designed to only be called from code with a lock on the TaskScheduler
- * (e.g. its event handlers). It should not be called from other threads.
+ * @param sched the ClusterScheduler associated with the TaskSetManager
+ * @param taskSet the TaskSet to manage scheduling for
+ * @param maxTaskFailures if any particular task fails more than this number of times, the entire
+ * task set will be aborted
*/
-private[spark] trait TaskSetManager extends Schedulable {
- def schedulableQueue = null
-
- def schedulingMode = SchedulingMode.NONE
-
- def taskSet: TaskSet
+private[spark] class TaskSetManager(
+ sched: TaskSchedulerImpl,
+ val taskSet: TaskSet,
+ val maxTaskFailures: Int,
+ clock: Clock = SystemClock)
+ extends Schedulable with Logging
+{
+ val conf = sched.sc.conf
+
+ // CPUs to request per task
+ val CPUS_PER_TASK = conf.get("spark.task.cpus", "1").toInt
+
+ // Quantile of tasks at which to start speculation
+ val SPECULATION_QUANTILE = conf.get("spark.speculation.quantile", "0.75").toDouble
+ val SPECULATION_MULTIPLIER = conf.get("spark.speculation.multiplier", "1.5").toDouble
+
+ // Serializer for closures and tasks.
+ val env = SparkEnv.get
+ val ser = env.closureSerializer.newInstance()
+
+ val tasks = taskSet.tasks
+ val numTasks = tasks.length
+ val copiesRunning = new Array[Int](numTasks)
+ val successful = new Array[Boolean](numTasks)
+ val numFailures = new Array[Int](numTasks)
+ val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil)
+ var tasksSuccessful = 0
+
+ var weight = 1
+ var minShare = 0
+ var priority = taskSet.priority
+ var stageId = taskSet.stageId
+ var name = "TaskSet_"+taskSet.stageId.toString
+ 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
+ // ArrayBuffer and removed from the end. This makes it faster to detect
+ // tasks that repeatedly fail because whenever a task failed, it is put
+ // back at the head of the stack. They are also only cleaned up lazily;
+ // when a task is launched, it remains in all the pending lists except
+ // the one that it was launched from, but gets removed from them later.
+ private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
+
+ // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
+ // but at host level.
+ private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]]
+
+ // Set of pending tasks for each rack -- similar to the above.
+ private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
+
+ // Set containing pending tasks with no locality preferences.
+ val pendingTasksWithNoPrefs = new ArrayBuffer[Int]
+
+ // Set containing all pending tasks (also used as a stack, as above).
+ val allPendingTasks = new ArrayBuffer[Int]
+
+ // Tasks that can be speculated. Since these will be a small fraction of total
+ // tasks, we'll just hold them in a HashSet.
+ val speculatableTasks = new HashSet[Int]
+
+ // Task index, start and finish time for each task attempt (indexed by task ID)
+ val taskInfos = new HashMap[Long, TaskInfo]
+
+ // How frequently to reprint duplicate exceptions in full, in milliseconds
+ val EXCEPTION_PRINT_INTERVAL =
+ conf.get("spark.logging.exceptionPrintInterval", "10000").toLong
+
+ // Map of recent exceptions (identified by string representation and top stack frame) to
+ // duplicate count (how many times the same exception has appeared) and time the full exception
+ // was printed. This should ideally be an LRU map that can drop old exceptions automatically.
+ val recentExceptions = HashMap[String, (Int, Long)]()
+
+ // Figure out the current map output tracker epoch and set it on all tasks
+ val epoch = sched.mapOutputTracker.getEpoch
+ logDebug("Epoch for " + taskSet + ": " + epoch)
+ for (t <- tasks) {
+ t.epoch = epoch
+ }
+
+ // Add all our tasks to the pending lists. We do this in reverse order
+ // of task index so that tasks with low indices get launched first.
+ for (i <- (0 until numTasks).reverse) {
+ addPendingTask(i)
+ }
+
+ // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling
+ val myLocalityLevels = computeValidLocalityLevels()
+ val localityWaits = myLocalityLevels.map(getLocalityWait) // Time to wait at each level
+
+ // Delay scheduling variables: we keep track of our current locality level and the time we
+ // last launched a task at that level, and move up a level when localityWaits[curLevel] expires.
+ // We then move down if we manage to launch a "more local" task.
+ var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels
+ var lastLaunchTime = clock.getTime() // Time we last launched a task at this level
+
+ override def schedulableQueue = null
+
+ override def schedulingMode = SchedulingMode.NONE
+
+ /**
+ * Add a task to all the pending-task lists that it should be on. If readding is set, we are
+ * re-adding the task so only include it in each list if it's not already there.
+ */
+ private def addPendingTask(index: Int, readding: Boolean = false) {
+ // Utility method that adds `index` to a list only if readding=false or it's not already there
+ def addTo(list: ArrayBuffer[Int]) {
+ if (!readding || !list.contains(index)) {
+ list += index
+ }
+ }
+
+ var hadAliveLocations = false
+ for (loc <- tasks(index).preferredLocations) {
+ for (execId <- loc.executorId) {
+ if (sched.isExecutorAlive(execId)) {
+ addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer))
+ hadAliveLocations = true
+ }
+ }
+ if (sched.hasExecutorsAliveOnHost(loc.host)) {
+ addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer))
+ for (rack <- sched.getRackForHost(loc.host)) {
+ addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer))
+ }
+ hadAliveLocations = true
+ }
+ }
+
+ if (!hadAliveLocations) {
+ // Even though the task might've had preferred locations, all of those hosts or executors
+ // are dead; put it in the no-prefs list so we can schedule it elsewhere right away.
+ addTo(pendingTasksWithNoPrefs)
+ }
+
+ if (!readding) {
+ allPendingTasks += index // No point scanning this whole list to find the old task there
+ }
+ }
+
+ /**
+ * Return the pending tasks list for a given executor ID, or an empty list if
+ * there is no map entry for that host
+ */
+ private def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = {
+ pendingTasksForExecutor.getOrElse(executorId, ArrayBuffer())
+ }
+
+ /**
+ * Return the pending tasks list for a given host, or an empty list if
+ * there is no map entry for that host
+ */
+ private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = {
+ pendingTasksForHost.getOrElse(host, ArrayBuffer())
+ }
+
+ /**
+ * Return the pending rack-local task list for a given rack, or an empty list if
+ * there is no map entry for that rack
+ */
+ private def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = {
+ pendingTasksForRack.getOrElse(rack, ArrayBuffer())
+ }
+
+ /**
+ * Dequeue a pending task from the given list and return its index.
+ * Return None if the list is empty.
+ * This method also cleans up any tasks in the list that have already
+ * been launched, since we want that to happen lazily.
+ */
+ private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = {
+ while (!list.isEmpty) {
+ val index = list.last
+ list.trimEnd(1)
+ if (copiesRunning(index) == 0 && !successful(index)) {
+ return Some(index)
+ }
+ }
+ return None
+ }
+
+ /** Check whether a task is currently running an attempt on a given host */
+ private def hasAttemptOnHost(taskIndex: Int, host: String): Boolean = {
+ !taskAttempts(taskIndex).exists(_.host == host)
+ }
+
+ /**
+ * Return a speculative task for a given executor if any are available. The task should not have
+ * an attempt running on this host, in case the host is slow. In addition, the task should meet
+ * the given locality constraint.
+ */
+ private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value)
+ : Option[(Int, TaskLocality.Value)] =
+ {
+ 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
+ // on multiple nodes when we replicate cached blocks, as in Spark Streaming
+ for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
+ val prefs = tasks(index).preferredLocations
+ val executors = prefs.flatMap(_.executorId)
+ if (prefs.size == 0 || executors.contains(execId)) {
+ speculatableTasks -= index
+ return Some((index, TaskLocality.PROCESS_LOCAL))
+ }
+ }
+
+ // Check for node-local tasks
+ if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) {
+ for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
+ val locations = tasks(index).preferredLocations.map(_.host)
+ if (locations.contains(host)) {
+ speculatableTasks -= index
+ return Some((index, TaskLocality.NODE_LOCAL))
+ }
+ }
+ }
+ // Check for rack-local tasks
+ if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
+ for (rack <- sched.getRackForHost(host)) {
+ for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
+ val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost)
+ if (racks.contains(rack)) {
+ speculatableTasks -= index
+ return Some((index, TaskLocality.RACK_LOCAL))
+ }
+ }
+ }
+ }
+
+ // Check for non-local tasks
+ if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
+ for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
+ speculatableTasks -= index
+ return Some((index, TaskLocality.ANY))
+ }
+ }
+ }
+
+ return None
+ }
+
+ /**
+ * Dequeue a pending task for a given node and return its index and locality level.
+ * Only search for tasks matching the given locality constraint.
+ */
+ private def findTask(execId: String, host: String, locality: TaskLocality.Value)
+ : Option[(Int, TaskLocality.Value)] =
+ {
+ for (index <- findTaskFromList(getPendingTasksForExecutor(execId))) {
+ return Some((index, TaskLocality.PROCESS_LOCAL))
+ }
+
+ if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) {
+ for (index <- findTaskFromList(getPendingTasksForHost(host))) {
+ return Some((index, TaskLocality.NODE_LOCAL))
+ }
+ }
+
+ if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
+ for {
+ rack <- sched.getRackForHost(host)
+ index <- findTaskFromList(getPendingTasksForRack(rack))
+ } {
+ return Some((index, TaskLocality.RACK_LOCAL))
+ }
+ }
+
+ // Look for no-pref tasks after rack-local tasks since they can run anywhere.
+ for (index <- findTaskFromList(pendingTasksWithNoPrefs)) {
+ return Some((index, TaskLocality.PROCESS_LOCAL))
+ }
+
+ if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
+ for (index <- findTaskFromList(allPendingTasks)) {
+ return Some((index, TaskLocality.ANY))
+ }
+ }
+
+ // Finally, if all else has failed, find a speculative task
+ return findSpeculativeTask(execId, host, locality)
+ }
+
+ /**
+ * Respond to an offer of a single executor from the scheduler by finding a task
+ */
def resourceOffer(
execId: String,
host: String,
availableCpus: Int,
maxLocality: TaskLocality.TaskLocality)
- : Option[TaskDescription]
+ : Option[TaskDescription] =
+ {
+ if (tasksSuccessful < numTasks && availableCpus >= CPUS_PER_TASK) {
+ val curTime = clock.getTime()
+
+ var allowedLocality = getAllowedLocalityLevel(curTime)
+ if (allowedLocality > maxLocality) {
+ allowedLocality = maxLocality // We're not allowed to search for farther-away tasks
+ }
+
+ findTask(execId, host, allowedLocality) match {
+ case Some((index, taskLocality)) => {
+ // Found a task; do some bookkeeping and return a task description
+ 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 executor %s: %s (%s)".format(
+ taskSet.id, index, taskId, execId, host, taskLocality))
+ // Do various bookkeeping
+ copiesRunning(index) += 1
+ val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality)
+ taskInfos(taskId) = info
+ taskAttempts(index) = info :: taskAttempts(index)
+ // Update our locality level for delay scheduling
+ currentLocalityIndex = getLocalityIndex(taskLocality)
+ lastLaunchTime = curTime
+ // Serialize and return the task
+ val startTime = clock.getTime()
+ // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here
+ // we assume the task can be serialized without exceptions.
+ val serializedTask = Task.serializeWithDependencies(
+ task, sched.sc.addedFiles, sched.sc.addedJars, ser)
+ val timeTaken = clock.getTime() - startTime
+ 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)
+ if (taskAttempts(index).size == 1)
+ taskStarted(task,info)
+ return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask))
+ }
+ case _ =>
+ }
+ }
+ return None
+ }
+
+ /**
+ * Get the level we can launch tasks according to delay scheduling, based on current wait time.
+ */
+ private def getAllowedLocalityLevel(curTime: Long): TaskLocality.TaskLocality = {
+ while (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex) &&
+ currentLocalityIndex < myLocalityLevels.length - 1)
+ {
+ // Jump to the next locality level, and remove our waiting time for the current one since
+ // we don't want to count it again on the next one
+ lastLaunchTime += localityWaits(currentLocalityIndex)
+ currentLocalityIndex += 1
+ }
+ myLocalityLevels(currentLocalityIndex)
+ }
+
+ /**
+ * Find the index in myLocalityLevels for a given locality. This is also designed to work with
+ * localities that are not in myLocalityLevels (in case we somehow get those) by returning the
+ * next-biggest level we have. Uses the fact that the last value in myLocalityLevels is ANY.
+ */
+ def getLocalityIndex(locality: TaskLocality.TaskLocality): Int = {
+ var index = 0
+ while (locality > myLocalityLevels(index)) {
+ index += 1
+ }
+ index
+ }
+
+ private def taskStarted(task: Task[_], info: TaskInfo) {
+ sched.dagScheduler.taskStarted(task, info)
+ }
+
+ def handleTaskGettingResult(tid: Long) = {
+ val info = taskInfos(tid)
+ info.markGettingResult()
+ sched.dagScheduler.taskGettingResult(tasks(info.index), info)
+ }
+
+ /**
+ * Marks the task as successful and notifies the DAGScheduler that a task has ended.
+ */
+ def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = {
+ val info = taskInfos(tid)
+ val index = info.index
+ info.markSuccessful()
+ removeRunningTask(tid)
+ if (!successful(index)) {
+ logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format(
+ tid, info.duration, info.host, tasksSuccessful, numTasks))
+ sched.dagScheduler.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("Ignorning task-finished event for TID " + tid + " because task " +
+ index + " has already completed successfully")
+ }
+ }
+
+ /**
+ * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the
+ * DAG Scheduler.
+ */
+ def handleFailedTask(tid: Long, state: TaskState, reason: Option[TaskEndReason]) {
+ val info = taskInfos(tid)
+ if (info.failed) {
+ return
+ }
+ removeRunningTask(tid)
+ val index = info.index
+ info.markFailed()
+ var failureReason = "unknown"
+ if (!successful(index)) {
+ logWarning("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.
+ reason.foreach {
+ case fetchFailed: FetchFailed =>
+ logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress)
+ sched.dagScheduler.taskEnded(tasks(index), fetchFailed, null, null, info, null)
+ successful(index) = true
+ tasksSuccessful += 1
+ sched.taskSetFinished(this)
+ removeAllRunningTasks()
+ return
+
+ case TaskKilled =>
+ logWarning("Task %d was killed.".format(tid))
+ sched.dagScheduler.taskEnded(tasks(index), reason.get, null, null, info, null)
+ return
+
+ case ef: ExceptionFailure =>
+ sched.dagScheduler.taskEnded(
+ tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
+ if (ef.className == classOf[NotSerializableException].getName()) {
+ // If the task result wasn't rerializable, there's no point in trying to re-execute it.
+ logError("Task %s:%s had a not serializable result: %s; not retrying".format(
+ taskSet.id, index, ef.description))
+ abort("Task %s:%s had a not serializable result: %s".format(
+ taskSet.id, index, ef.description))
+ return
+ }
+ val key = ef.description
+ failureReason = "Exception failure: %s".format(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 {
+ recentExceptions(key) = (0, now)
+ (true, 0)
+ }
+ }
+ if (printFull) {
+ val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString))
+ logWarning("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 TaskResultLost =>
+ failureReason = "Lost result for TID %s on host %s".format(tid, info.host)
+ logWarning(failureReason)
+ sched.dagScheduler.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)
+ if (state != TaskState.KILLED) {
+ numFailures(index) += 1
+ if (numFailures(index) >= maxTaskFailures) {
+ logError("Task %s:%d failed %d times; aborting job".format(
+ taskSet.id, index, maxTaskFailures))
+ abort("Task %s:%d failed %d times (most recent failure: %s)".format(
+ taskSet.id, index, maxTaskFailures, failureReason))
+ }
+ }
+ } else {
+ logInfo("Ignoring task-lost event for TID " + tid +
+ " because task " + index + " is already finished")
+ }
+ }
+
+ def abort(message: String) {
+ // TODO: Kill running tasks if we were not terminated due to a Mesos error
+ sched.dagScheduler.taskSetFailed(taskSet, message)
+ removeAllRunningTasks()
+ sched.taskSetFinished(this)
+ }
+
+ /** 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
+ }
+
+ private[scheduler] def removeAllRunningTasks() {
+ val numRunningTasks = runningTasksSet.size
+ runningTasksSet.clear()
+ if (parent != null) {
+ parent.decreaseRunningTasks(numRunningTasks)
+ }
+ runningTasks = 0
+ }
+
+ override def getSchedulableByName(name: String): Schedulable = {
+ return null
+ }
+
+ override def addSchedulable(schedulable: Schedulable) {}
+
+ override def removeSchedulable(schedulable: Schedulable) {}
+
+ override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
+ var sortedTaskSetQueue = ArrayBuffer[TaskSetManager](this)
+ sortedTaskSetQueue += this
+ return sortedTaskSetQueue
+ }
+
+ /** Called by TaskScheduler when an executor is lost so we can re-enqueue our tasks */
+ override def executorLost(execId: String, host: String) {
+ logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id)
+
+ // Re-enqueue pending tasks for this host based on the status of the cluster -- for example, a
+ // task that used to have locations on only this host might now go to the no-prefs list. Note
+ // that it's okay if we add a task to the same queue twice (if it had multiple preferred
+ // locations), because findTaskFromList will skip already-running tasks.
+ for (index <- getPendingTasksForExecutor(execId)) {
+ addPendingTask(index, readding=true)
+ }
+ for (index <- getPendingTasksForHost(host)) {
+ addPendingTask(index, readding=true)
+ }
+
+ // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage
+ if (tasks(0).isInstanceOf[ShuffleMapTask]) {
+ for ((tid, info) <- taskInfos if info.executorId == execId) {
+ val index = taskInfos(tid).index
+ if (successful(index)) {
+ successful(index) = false
+ copiesRunning(index) -= 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.
+ sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info, null)
+ }
+ }
+ }
+ // Also re-enqueue any tasks that were running on the node
+ for ((tid, info) <- taskInfos if info.running && info.executorId == execId) {
+ handleFailedTask(tid, TaskState.KILLED, None)
+ }
+ }
+
+ /**
+ * Check for tasks to be speculated and return true if there are any. This is called periodically
+ * by the TaskScheduler.
+ *
+ * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that
+ * we don't scan the whole task set. It might also help to make this sorted by launch time.
+ */
+ override def checkSpeculatableTasks(): Boolean = {
+ // Can't speculate if we only have one task, or if all tasks have finished.
+ 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 (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 * 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 (!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(
+ taskSet.id, index, info.host, threshold))
+ speculatableTasks += index
+ foundTasks = true
+ }
+ }
+ }
+ return foundTasks
+ }
+
+ private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
+ val defaultWait = conf.get("spark.locality.wait", "3000")
+ level match {
+ case TaskLocality.PROCESS_LOCAL =>
+ conf.get("spark.locality.wait.process", defaultWait).toLong
+ case TaskLocality.NODE_LOCAL =>
+ conf.get("spark.locality.wait.node", defaultWait).toLong
+ case TaskLocality.RACK_LOCAL =>
+ conf.get("spark.locality.wait.rack", defaultWait).toLong
+ case TaskLocality.ANY =>
+ 0L
+ }
+ }
- def error(message: String)
+ /**
+ * Compute the locality levels used in this TaskSet. Assumes that all tasks have already been
+ * added to queues using addPendingTask.
+ */
+ private def computeValidLocalityLevels(): Array[TaskLocality.TaskLocality] = {
+ import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY}
+ val levels = new ArrayBuffer[TaskLocality.TaskLocality]
+ if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0) {
+ levels += PROCESS_LOCAL
+ }
+ if (!pendingTasksForHost.isEmpty && getLocalityWait(NODE_LOCAL) != 0) {
+ levels += NODE_LOCAL
+ }
+ if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0) {
+ levels += RACK_LOCAL
+ }
+ levels += ANY
+ logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", "))
+ levels.toArray
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
index 938f62883a..ba6bab3f91 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/WorkerOffer.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
/**
* Represents free resources available on an executor.
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
deleted file mode 100644
index 4c5eca8537..0000000000
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
+++ /dev/null
@@ -1,712 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.scheduler.cluster
-
-import java.io.NotSerializableException
-import java.util.Arrays
-
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
-import scala.math.max
-import scala.math.min
-
-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 org.apache.spark.util.{SystemClock, Clock}
-
-
-/**
- * Schedules the tasks within a single TaskSet in the ClusterScheduler. This class keeps track of
- * the status of each task, retries tasks if they fail (up to a limited number of times), and
- * handles locality-aware scheduling for this TaskSet via delay scheduling. The main interfaces
- * to it are resourceOffer, which asks the TaskSet whether it wants to run a task on one node,
- * and statusUpdate, which tells it that one of its tasks changed state (e.g. finished).
- *
- * THREADING: This class is designed to only be called from code with a lock on the
- * ClusterScheduler (e.g. its event handlers). It should not be called from other threads.
- */
-private[spark] class ClusterTaskSetManager(
- sched: ClusterScheduler,
- val taskSet: TaskSet,
- clock: Clock = SystemClock)
- extends TaskSetManager
- with Logging
-{
- // CPUs to request per task
- val CPUS_PER_TASK = System.getProperty("spark.task.cpus", "1").toInt
-
- // Maximum times a task is allowed to fail before failing the job
- val MAX_TASK_FAILURES = System.getProperty("spark.task.maxFailures", "4").toInt
-
- // Quantile of tasks at which to start speculation
- val SPECULATION_QUANTILE = System.getProperty("spark.speculation.quantile", "0.75").toDouble
- val SPECULATION_MULTIPLIER = System.getProperty("spark.speculation.multiplier", "1.5").toDouble
-
- // Serializer for closures and tasks.
- val env = SparkEnv.get
- val ser = env.closureSerializer.newInstance()
-
- val tasks = taskSet.tasks
- val numTasks = tasks.length
- val copiesRunning = new Array[Int](numTasks)
- val successful = new Array[Boolean](numTasks)
- val numFailures = new Array[Int](numTasks)
- val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil)
- var tasksSuccessful = 0
-
- var weight = 1
- var minShare = 0
- var priority = taskSet.priority
- var stageId = taskSet.stageId
- var name = "TaskSet_"+taskSet.stageId.toString
- 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
- // ArrayBuffer and removed from the end. This makes it faster to detect
- // tasks that repeatedly fail because whenever a task failed, it is put
- // back at the head of the stack. They are also only cleaned up lazily;
- // when a task is launched, it remains in all the pending lists except
- // the one that it was launched from, but gets removed from them later.
- private val pendingTasksForExecutor = new HashMap[String, ArrayBuffer[Int]]
-
- // Set of pending tasks for each host. Similar to pendingTasksForExecutor,
- // but at host level.
- private val pendingTasksForHost = new HashMap[String, ArrayBuffer[Int]]
-
- // Set of pending tasks for each rack -- similar to the above.
- private val pendingTasksForRack = new HashMap[String, ArrayBuffer[Int]]
-
- // Set containing pending tasks with no locality preferences.
- val pendingTasksWithNoPrefs = new ArrayBuffer[Int]
-
- // Set containing all pending tasks (also used as a stack, as above).
- val allPendingTasks = new ArrayBuffer[Int]
-
- // Tasks that can be speculated. Since these will be a small fraction of total
- // tasks, we'll just hold them in a HashSet.
- val speculatableTasks = new HashSet[Int]
-
- // Task index, start and finish time for each task attempt (indexed by task ID)
- val taskInfos = new HashMap[Long, TaskInfo]
-
- // Did the TaskSet fail?
- var failed = false
- var causeOfFailure = ""
-
- // How frequently to reprint duplicate exceptions in full, in milliseconds
- val EXCEPTION_PRINT_INTERVAL =
- System.getProperty("spark.logging.exceptionPrintInterval", "10000").toLong
-
- // Map of recent exceptions (identified by string representation and top stack frame) to
- // duplicate count (how many times the same exception has appeared) and time the full exception
- // was printed. This should ideally be an LRU map that can drop old exceptions automatically.
- val recentExceptions = HashMap[String, (Int, Long)]()
-
- // Figure out the current map output tracker epoch and set it on all tasks
- val epoch = sched.mapOutputTracker.getEpoch
- logDebug("Epoch for " + taskSet + ": " + epoch)
- for (t <- tasks) {
- t.epoch = epoch
- }
-
- // Add all our tasks to the pending lists. We do this in reverse order
- // of task index so that tasks with low indices get launched first.
- for (i <- (0 until numTasks).reverse) {
- addPendingTask(i)
- }
-
- // Figure out which locality levels we have in our TaskSet, so we can do delay scheduling
- val myLocalityLevels = computeValidLocalityLevels()
- val localityWaits = myLocalityLevels.map(getLocalityWait) // Time to wait at each level
-
- // Delay scheduling variables: we keep track of our current locality level and the time we
- // last launched a task at that level, and move up a level when localityWaits[curLevel] expires.
- // We then move down if we manage to launch a "more local" task.
- var currentLocalityIndex = 0 // Index of our current locality level in validLocalityLevels
- var lastLaunchTime = clock.getTime() // Time we last launched a task at this level
-
- /**
- * Add a task to all the pending-task lists that it should be on. If readding is set, we are
- * re-adding the task so only include it in each list if it's not already there.
- */
- private def addPendingTask(index: Int, readding: Boolean = false) {
- // Utility method that adds `index` to a list only if readding=false or it's not already there
- def addTo(list: ArrayBuffer[Int]) {
- if (!readding || !list.contains(index)) {
- list += index
- }
- }
-
- var hadAliveLocations = false
- for (loc <- tasks(index).preferredLocations) {
- for (execId <- loc.executorId) {
- if (sched.isExecutorAlive(execId)) {
- addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer))
- hadAliveLocations = true
- }
- }
- if (sched.hasExecutorsAliveOnHost(loc.host)) {
- addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer))
- for (rack <- sched.getRackForHost(loc.host)) {
- addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer))
- }
- hadAliveLocations = true
- }
- }
-
- if (!hadAliveLocations) {
- // Even though the task might've had preferred locations, all of those hosts or executors
- // are dead; put it in the no-prefs list so we can schedule it elsewhere right away.
- addTo(pendingTasksWithNoPrefs)
- }
-
- if (!readding) {
- allPendingTasks += index // No point scanning this whole list to find the old task there
- }
- }
-
- /**
- * Return the pending tasks list for a given executor ID, or an empty list if
- * there is no map entry for that host
- */
- private def getPendingTasksForExecutor(executorId: String): ArrayBuffer[Int] = {
- pendingTasksForExecutor.getOrElse(executorId, ArrayBuffer())
- }
-
- /**
- * Return the pending tasks list for a given host, or an empty list if
- * there is no map entry for that host
- */
- private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = {
- pendingTasksForHost.getOrElse(host, ArrayBuffer())
- }
-
- /**
- * Return the pending rack-local task list for a given rack, or an empty list if
- * there is no map entry for that rack
- */
- private def getPendingTasksForRack(rack: String): ArrayBuffer[Int] = {
- pendingTasksForRack.getOrElse(rack, ArrayBuffer())
- }
-
- /**
- * Dequeue a pending task from the given list and return its index.
- * Return None if the list is empty.
- * This method also cleans up any tasks in the list that have already
- * been launched, since we want that to happen lazily.
- */
- private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = {
- while (!list.isEmpty) {
- val index = list.last
- list.trimEnd(1)
- if (copiesRunning(index) == 0 && !successful(index)) {
- return Some(index)
- }
- }
- return None
- }
-
- /** Check whether a task is currently running an attempt on a given host */
- private def hasAttemptOnHost(taskIndex: Int, host: String): Boolean = {
- !taskAttempts(taskIndex).exists(_.host == host)
- }
-
- /**
- * Return a speculative task for a given executor if any are available. The task should not have
- * an attempt running on this host, in case the host is slow. In addition, the task should meet
- * the given locality constraint.
- */
- private def findSpeculativeTask(execId: String, host: String, locality: TaskLocality.Value)
- : Option[(Int, TaskLocality.Value)] =
- {
- 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
- // on multiple nodes when we replicate cached blocks, as in Spark Streaming
- for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
- val prefs = tasks(index).preferredLocations
- val executors = prefs.flatMap(_.executorId)
- if (prefs.size == 0 || executors.contains(execId)) {
- speculatableTasks -= index
- return Some((index, TaskLocality.PROCESS_LOCAL))
- }
- }
-
- // Check for node-local tasks
- if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) {
- for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
- val locations = tasks(index).preferredLocations.map(_.host)
- if (locations.contains(host)) {
- speculatableTasks -= index
- return Some((index, TaskLocality.NODE_LOCAL))
- }
- }
- }
-
- // Check for rack-local tasks
- if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
- for (rack <- sched.getRackForHost(host)) {
- for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
- val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost)
- if (racks.contains(rack)) {
- speculatableTasks -= index
- return Some((index, TaskLocality.RACK_LOCAL))
- }
- }
- }
- }
-
- // Check for non-local tasks
- if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
- for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) {
- speculatableTasks -= index
- return Some((index, TaskLocality.ANY))
- }
- }
- }
-
- return None
- }
-
- /**
- * Dequeue a pending task for a given node and return its index and locality level.
- * Only search for tasks matching the given locality constraint.
- */
- private def findTask(execId: String, host: String, locality: TaskLocality.Value)
- : Option[(Int, TaskLocality.Value)] =
- {
- for (index <- findTaskFromList(getPendingTasksForExecutor(execId))) {
- return Some((index, TaskLocality.PROCESS_LOCAL))
- }
-
- if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) {
- for (index <- findTaskFromList(getPendingTasksForHost(host))) {
- return Some((index, TaskLocality.NODE_LOCAL))
- }
- }
-
- if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) {
- for {
- rack <- sched.getRackForHost(host)
- index <- findTaskFromList(getPendingTasksForRack(rack))
- } {
- return Some((index, TaskLocality.RACK_LOCAL))
- }
- }
-
- // Look for no-pref tasks after rack-local tasks since they can run anywhere.
- for (index <- findTaskFromList(pendingTasksWithNoPrefs)) {
- return Some((index, TaskLocality.PROCESS_LOCAL))
- }
-
- if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) {
- for (index <- findTaskFromList(allPendingTasks)) {
- return Some((index, TaskLocality.ANY))
- }
- }
-
- // Finally, if all else has failed, find a speculative task
- return findSpeculativeTask(execId, host, locality)
- }
-
- /**
- * Respond to an offer of a single executor from the scheduler by finding a task
- */
- override def resourceOffer(
- execId: String,
- host: String,
- availableCpus: Int,
- maxLocality: TaskLocality.TaskLocality)
- : Option[TaskDescription] =
- {
- if (tasksSuccessful < numTasks && availableCpus >= CPUS_PER_TASK) {
- val curTime = clock.getTime()
-
- var allowedLocality = getAllowedLocalityLevel(curTime)
- if (allowedLocality > maxLocality) {
- allowedLocality = maxLocality // We're not allowed to search for farther-away tasks
- }
-
- findTask(execId, host, allowedLocality) match {
- case Some((index, taskLocality)) => {
- // Found a task; do some bookkeeping and return a task description
- 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 executor %s: %s (%s)".format(
- taskSet.id, index, taskId, execId, host, taskLocality))
- // Do various bookkeeping
- copiesRunning(index) += 1
- val info = new TaskInfo(taskId, index, curTime, execId, host, taskLocality)
- taskInfos(taskId) = info
- taskAttempts(index) = info :: taskAttempts(index)
- // Update our locality level for delay scheduling
- currentLocalityIndex = getLocalityIndex(taskLocality)
- lastLaunchTime = curTime
- // Serialize and return the task
- val startTime = clock.getTime()
- // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here
- // we assume the task can be serialized without exceptions.
- val serializedTask = Task.serializeWithDependencies(
- task, sched.sc.addedFiles, sched.sc.addedJars, ser)
- val timeTaken = clock.getTime() - startTime
- 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)
- if (taskAttempts(index).size == 1)
- taskStarted(task,info)
- return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask))
- }
- case _ =>
- }
- }
- return None
- }
-
- /**
- * Get the level we can launch tasks according to delay scheduling, based on current wait time.
- */
- private def getAllowedLocalityLevel(curTime: Long): TaskLocality.TaskLocality = {
- while (curTime - lastLaunchTime >= localityWaits(currentLocalityIndex) &&
- currentLocalityIndex < myLocalityLevels.length - 1)
- {
- // Jump to the next locality level, and remove our waiting time for the current one since
- // we don't want to count it again on the next one
- lastLaunchTime += localityWaits(currentLocalityIndex)
- currentLocalityIndex += 1
- }
- myLocalityLevels(currentLocalityIndex)
- }
-
- /**
- * Find the index in myLocalityLevels for a given locality. This is also designed to work with
- * localities that are not in myLocalityLevels (in case we somehow get those) by returning the
- * next-biggest level we have. Uses the fact that the last value in myLocalityLevels is ANY.
- */
- def getLocalityIndex(locality: TaskLocality.TaskLocality): Int = {
- var index = 0
- while (locality > myLocalityLevels(index)) {
- index += 1
- }
- index
- }
-
- private def taskStarted(task: Task[_], info: TaskInfo) {
- sched.dagScheduler.taskStarted(task, info)
- }
-
- def handleTaskGettingResult(tid: Long) = {
- val info = taskInfos(tid)
- info.markGettingResult()
- sched.dagScheduler.taskGettingResult(tasks(info.index), info)
- }
-
- /**
- * Marks the task as successful and notifies the DAGScheduler that a task has ended.
- */
- def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]) = {
- val info = taskInfos(tid)
- val index = info.index
- info.markSuccessful()
- removeRunningTask(tid)
- if (!successful(index)) {
- logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format(
- tid, info.duration, info.host, tasksSuccessful, numTasks))
- sched.dagScheduler.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("Ignorning task-finished event for TID " + tid + " because task " +
- index + " has already completed successfully")
- }
- }
-
- /**
- * Marks the task as failed, re-adds it to the list of pending tasks, and notifies the
- * DAG Scheduler.
- */
- def handleFailedTask(tid: Long, state: TaskState, reason: Option[TaskEndReason]) {
- val info = taskInfos(tid)
- if (info.failed) {
- return
- }
- removeRunningTask(tid)
- val index = info.index
- info.markFailed()
- if (!successful(index)) {
- logWarning("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.
- reason.foreach {
- case fetchFailed: FetchFailed =>
- logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress)
- sched.dagScheduler.taskEnded(tasks(index), fetchFailed, null, null, info, null)
- successful(index) = true
- tasksSuccessful += 1
- sched.taskSetFinished(this)
- removeAllRunningTasks()
- return
-
- case TaskKilled =>
- logWarning("Task %d was killed.".format(tid))
- sched.dagScheduler.taskEnded(tasks(index), reason.get, null, null, info, null)
- return
-
- case ef: ExceptionFailure =>
- sched.dagScheduler.taskEnded(tasks(index), ef, null, null, info, ef.metrics.getOrElse(null))
- if (ef.className == classOf[NotSerializableException].getName()) {
- // If the task result wasn't serializable, there's no point in trying to re-execute it.
- logError("Task %s:%s had a not serializable result: %s; not retrying".format(
- taskSet.id, index, ef.description))
- abort("Task %s:%s had a not serializable result: %s".format(
- taskSet.id, index, ef.description))
- return
- }
- 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 {
- recentExceptions(key) = (0, now)
- (true, 0)
- }
- }
- if (printFull) {
- val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString))
- logWarning("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 TaskResultLost =>
- logWarning("Lost result for TID %s on host %s".format(tid, info.host))
- sched.dagScheduler.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)
- 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(
- taskSet.id, index, MAX_TASK_FAILURES))
- abort("Task %s:%d failed more than %d times".format(taskSet.id, index, MAX_TASK_FAILURES))
- }
- }
- } else {
- logInfo("Ignoring task-lost event for TID " + tid +
- " because task " + index + " is already finished")
- }
- }
-
- override def error(message: String) {
- // Save the error message
- abort("Error: " + message)
- }
-
- def abort(message: String) {
- failed = true
- causeOfFailure = message
- // TODO: Kill running tasks if we were not terminated due to a Mesos error
- sched.dagScheduler.taskSetFailed(taskSet, message)
- removeAllRunningTasks()
- sched.taskSetFinished(this)
- }
-
- /** 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
- }
-
- private def removeAllRunningTasks() {
- val numRunningTasks = runningTasksSet.size
- runningTasksSet.clear()
- if (parent != null) {
- parent.decreaseRunningTasks(numRunningTasks)
- }
- runningTasks = 0
- }
-
- override def getSchedulableByName(name: String): Schedulable = {
- return null
- }
-
- override def addSchedulable(schedulable: Schedulable) {}
-
- override def removeSchedulable(schedulable: Schedulable) {}
-
- override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
- var sortedTaskSetQueue = ArrayBuffer[TaskSetManager](this)
- sortedTaskSetQueue += this
- return sortedTaskSetQueue
- }
-
- /** Called by cluster scheduler when an executor is lost so we can re-enqueue our tasks */
- override def executorLost(execId: String, host: String) {
- logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id)
-
- // Re-enqueue pending tasks for this host based on the status of the cluster -- for example, a
- // task that used to have locations on only this host might now go to the no-prefs list. Note
- // that it's okay if we add a task to the same queue twice (if it had multiple preferred
- // locations), because findTaskFromList will skip already-running tasks.
- for (index <- getPendingTasksForExecutor(execId)) {
- addPendingTask(index, readding=true)
- }
- for (index <- getPendingTasksForHost(host)) {
- addPendingTask(index, readding=true)
- }
-
- // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage
- if (tasks(0).isInstanceOf[ShuffleMapTask]) {
- for ((tid, info) <- taskInfos if info.executorId == execId) {
- val index = taskInfos(tid).index
- if (successful(index)) {
- successful(index) = false
- copiesRunning(index) -= 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.
- sched.dagScheduler.taskEnded(tasks(index), Resubmitted, null, null, info, null)
- }
- }
- }
- // Also re-enqueue any tasks that were running on the node
- for ((tid, info) <- taskInfos if info.running && info.executorId == execId) {
- handleFailedTask(tid, TaskState.KILLED, None)
- }
- }
-
- /**
- * Check for tasks to be speculated and return true if there are any. This is called periodically
- * by the ClusterScheduler.
- *
- * TODO: To make this scale to large jobs, we need to maintain a list of running tasks, so that
- * we don't scan the whole task set. It might also help to make this sorted by launch time.
- */
- override def checkSpeculatableTasks(): Boolean = {
- // Can't speculate if we only have one task, or if all tasks have finished.
- 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 (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 * 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 (!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(
- taskSet.id, index, info.host, threshold))
- speculatableTasks += index
- foundTasks = true
- }
- }
- }
- return foundTasks
- }
-
- override def hasPendingTasks(): Boolean = {
- numTasks > 0 && tasksSuccessful < numTasks
- }
-
- private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
- val defaultWait = System.getProperty("spark.locality.wait", "3000")
- level match {
- case TaskLocality.PROCESS_LOCAL =>
- System.getProperty("spark.locality.wait.process", defaultWait).toLong
- case TaskLocality.NODE_LOCAL =>
- System.getProperty("spark.locality.wait.node", defaultWait).toLong
- case TaskLocality.RACK_LOCAL =>
- System.getProperty("spark.locality.wait.rack", defaultWait).toLong
- case TaskLocality.ANY =>
- 0L
- }
- }
-
- /**
- * Compute the locality levels used in this TaskSet. Assumes that all tasks have already been
- * added to queues using addPendingTask.
- */
- private def computeValidLocalityLevels(): Array[TaskLocality.TaskLocality] = {
- import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY}
- val levels = new ArrayBuffer[TaskLocality.TaskLocality]
- if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0) {
- levels += PROCESS_LOCAL
- }
- if (!pendingTasksForHost.isEmpty && getLocalityWait(NODE_LOCAL) != 0) {
- levels += NODE_LOCAL
- }
- if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0) {
- levels += RACK_LOCAL
- }
- levels += ANY
- logDebug("Valid locality levels for " + taskSet + ": " + levels.mkString(", "))
- levels.toArray
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index d0ba5bf55d..2f5bcafe40 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -20,18 +20,19 @@ package org.apache.spark.scheduler.cluster
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+import scala.concurrent.Await
+import scala.concurrent.duration._
import akka.actor._
-import akka.dispatch.Await
import akka.pattern.ask
-import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClientLifeCycleEvent}
-import akka.util.Duration
-import akka.util.duration._
+import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent}
import org.apache.spark.{SparkException, Logging, TaskState}
-import org.apache.spark.scheduler.TaskDescription
+import org.apache.spark.{Logging, SparkException, TaskState}
+import org.apache.spark.scheduler.{TaskSchedulerImpl, SchedulerBackend, SlaveLost, TaskDescription,
+ WorkerOffer}
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
-import org.apache.spark.util.Utils
+import org.apache.spark.util.{AkkaUtils, Utils}
/**
* A scheduler backend that waits for coarse grained executors to connect to it through Akka.
@@ -42,26 +43,28 @@ import org.apache.spark.util.Utils
* (spark.deploy.*).
*/
private[spark]
-class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem)
+class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: ActorSystem)
extends SchedulerBackend with Logging
{
// Use an atomic variable to track total number of cores in the cluster for simplicity and speed
var totalCoreCount = new AtomicInteger(0)
+ val conf = scheduler.sc.conf
+ private val timeout = AkkaUtils.askTimeout(conf)
class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor {
private val executorActor = new HashMap[String, ActorRef]
private val executorAddress = new HashMap[String, Address]
private val executorHost = new HashMap[String, String]
private val freeCores = new HashMap[String, Int]
- private val actorToExecutorId = new HashMap[ActorRef, String]
private val addressToExecutorId = new HashMap[Address, String]
override def preStart() {
// Listen for remote client disconnection events, since they don't go through Akka's watch()
- context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
// Periodically revive offers to allow delay scheduling to work
- val reviveInterval = System.getProperty("spark.scheduler.revive.interval", "1000").toLong
+ val reviveInterval = conf.get("spark.scheduler.revive.interval", "1000").toLong
+ import context.dispatcher
context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers)
}
@@ -73,12 +76,10 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
} else {
logInfo("Registered executor: " + sender + " with ID " + executorId)
sender ! RegisteredExecutor(sparkProperties)
- context.watch(sender)
executorActor(executorId) = sender
executorHost(executorId) = Utils.parseHostPort(hostPort)._1
freeCores(executorId) = cores
executorAddress(executorId) = sender.path.address
- actorToExecutorId(sender) = executorId
addressToExecutorId(sender.path.address) = executorId
totalCoreCount.addAndGet(cores)
makeOffers()
@@ -118,14 +119,9 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
removeExecutor(executorId, reason)
sender ! true
- case Terminated(actor) =>
- actorToExecutorId.get(actor).foreach(removeExecutor(_, "Akka actor terminated"))
+ case DisassociatedEvent(_, address, _) =>
+ addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated"))
- case RemoteClientDisconnected(transport, address) =>
- addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disconnected"))
-
- case RemoteClientShutdown(transport, address) =>
- addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client shutdown"))
}
// Make fake resource offers on all executors
@@ -153,7 +149,6 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
if (executorActor.contains(executorId)) {
logInfo("Executor " + executorId + " disconnected, so removing it")
val numCores = freeCores(executorId)
- actorToExecutorId -= executorActor(executorId)
addressToExecutorId -= executorAddress(executorId)
executorActor -= executorId
executorHost -= executorId
@@ -169,22 +164,16 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
override def start() {
val properties = new ArrayBuffer[(String, String)]
- val iterator = System.getProperties.entrySet.iterator
- while (iterator.hasNext) {
- val entry = iterator.next
- val (key, value) = (entry.getKey.toString, entry.getValue.toString)
+ for ((key, value) <- scheduler.sc.conf.getAll) {
if (key.startsWith("spark.") && !key.equals("spark.hostPort")) {
properties += ((key, value))
}
}
+ //TODO (prashant) send conf instead of properties
driverActor = actorSystem.actorOf(
Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME)
}
- private val timeout = {
- Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
- }
-
def stopExecutors() {
try {
if (driverActor != null) {
@@ -219,8 +208,10 @@ class CoarseGrainedSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Ac
driverActor ! KillTask(taskId, executorId)
}
- override def defaultParallelism() = Option(System.getProperty("spark.default.parallelism"))
- .map(_.toInt).getOrElse(math.max(totalCoreCount.get(), 2))
+ override def defaultParallelism(): Int = {
+ conf.getOption("spark.default.parallelism").map(_.toInt).getOrElse(
+ math.max(totalCoreCount.get(), 2))
+ }
// Called by subclasses when notified of a lost worker
def removeExecutor(executorId: String, reason: String) {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
index e000531a26..b44d1e43c8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala
@@ -19,10 +19,12 @@ package org.apache.spark.scheduler.cluster
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, FileSystem}
+
import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.scheduler.TaskSchedulerImpl
private[spark] class SimrSchedulerBackend(
- scheduler: ClusterScheduler,
+ scheduler: TaskSchedulerImpl,
sc: SparkContext,
driverFilePath: String)
extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
@@ -31,13 +33,13 @@ private[spark] class SimrSchedulerBackend(
val tmpPath = new Path(driverFilePath + "_tmp")
val filePath = new Path(driverFilePath)
- val maxCores = System.getProperty("spark.simr.executor.cores", "1").toInt
+ val maxCores = conf.get("spark.simr.executor.cores", "1").toInt
override def start() {
super.start()
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
- System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+ sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
val conf = new Configuration()
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 cefa970bb9..73fc37444e 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
@@ -17,14 +17,16 @@
package org.apache.spark.scheduler.cluster
+import scala.collection.mutable.HashMap
+
import org.apache.spark.{Logging, SparkContext}
import org.apache.spark.deploy.client.{Client, ClientListener}
import org.apache.spark.deploy.{Command, ApplicationDescription}
-import scala.collection.mutable.HashMap
+import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl}
import org.apache.spark.util.Utils
private[spark] class SparkDeploySchedulerBackend(
- scheduler: ClusterScheduler,
+ scheduler: TaskSchedulerImpl,
sc: SparkContext,
masters: Array[String],
appName: String)
@@ -36,23 +38,23 @@ private[spark] class SparkDeploySchedulerBackend(
var stopping = false
var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
- val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt
+ val maxCores = conf.getOption("spark.cores.max").map(_.toInt)
override def start() {
super.start()
// The endpoint for executors to talk to us
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
- System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+ conf.get("spark.driver.host"), conf.get("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
val command = Command(
"org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs)
val sparkHome = sc.getSparkHome().getOrElse(null)
- val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome,
+ val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, sparkHome,
"http://" + sc.ui.appUIAddress)
- client = new Client(sc.env.actorSystem, masters, appDesc, this)
+ client = new Client(sc.env.actorSystem, masters, appDesc, this, conf)
client.start()
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index cd521e0f2b..d46fceba89 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
@@ -30,7 +30,8 @@ import org.apache.mesos._
import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
import org.apache.spark.{SparkException, Logging, SparkContext, TaskState}
-import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
+import org.apache.spark.scheduler.TaskSchedulerImpl
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
/**
* A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds
@@ -43,7 +44,7 @@ import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedu
* remove this.
*/
private[spark] class CoarseMesosSchedulerBackend(
- scheduler: ClusterScheduler,
+ scheduler: TaskSchedulerImpl,
sc: SparkContext,
master: String,
appName: String)
@@ -61,7 +62,7 @@ private[spark] class CoarseMesosSchedulerBackend(
var driver: SchedulerDriver = null
// Maximum number of cores to acquire (TODO: we'll need more flexible controls here)
- val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt
+ val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt
// Cores we have acquired with each Mesos task ID
val coresByTaskId = new HashMap[Int, Int]
@@ -76,7 +77,7 @@ private[spark] class CoarseMesosSchedulerBackend(
"Spark home is not set; set it through the spark.home system " +
"property, the SPARK_HOME environment variable or the SparkContext constructor"))
- val extraCoresPerSlave = System.getProperty("spark.mesos.extra.cores", "0").toInt
+ val extraCoresPerSlave = conf.get("spark.mesos.extra.cores", "0").toInt
var nextMesosTaskId = 0
@@ -120,13 +121,13 @@ private[spark] class CoarseMesosSchedulerBackend(
}
val command = CommandInfo.newBuilder()
.setEnvironment(environment)
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
- System.getProperty("spark.driver.host"),
- System.getProperty("spark.driver.port"),
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+ conf.get("spark.driver.host"),
+ conf.get("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
- val uri = System.getProperty("spark.executor.uri")
+ val uri = conf.get("spark.executor.uri", null)
if (uri == null) {
- val runScript = new File(sparkHome, "spark-class").getCanonicalPath
+ val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath
command.setValue(
"\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format(
runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
@@ -135,7 +136,7 @@ private[spark] class CoarseMesosSchedulerBackend(
// glob the directory "correctly".
val basename = uri.split('/').last.split('.').head
command.setValue(
- "cd %s*; ./spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
+ "cd %s*; ./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d"
.format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores))
command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
}
@@ -176,7 +177,7 @@ private[spark] class CoarseMesosSchedulerBackend(
val slaveId = offer.getSlaveId.toString
val mem = getResource(offer.getResourcesList, "mem")
val cpus = getResource(offer.getResourcesList, "cpus").toInt
- if (totalCoresAcquired < maxCores && mem >= executorMemory && cpus >= 1 &&
+ if (totalCoresAcquired < maxCores && mem >= sc.executorMemory && cpus >= 1 &&
failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES &&
!slaveIdsWithExecutors.contains(slaveId)) {
// Launch an executor on the slave
@@ -192,7 +193,7 @@ private[spark] class CoarseMesosSchedulerBackend(
.setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave))
.setName("Task " + taskId)
.addResources(createResource("cpus", cpusToUse))
- .addResources(createResource("mem", executorMemory))
+ .addResources(createResource("mem", sc.executorMemory))
.build()
d.launchTasks(offer.getId, Collections.singletonList(task), filters)
} else {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 50cbc2ca92..ae8d527352 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -30,9 +30,8 @@ import org.apache.mesos._
import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _}
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.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost,
+ TaskDescription, TaskSchedulerImpl, WorkerOffer}
import org.apache.spark.util.Utils
/**
@@ -41,7 +40,7 @@ import org.apache.spark.util.Utils
* from multiple apps can run on different cores) and in time (a core can switch ownership).
*/
private[spark] class MesosSchedulerBackend(
- scheduler: ClusterScheduler,
+ scheduler: TaskSchedulerImpl,
sc: SparkContext,
master: String,
appName: String)
@@ -101,20 +100,20 @@ private[spark] class MesosSchedulerBackend(
}
val command = CommandInfo.newBuilder()
.setEnvironment(environment)
- val uri = System.getProperty("spark.executor.uri")
+ val uri = sc.conf.get("spark.executor.uri", null)
if (uri == null) {
- command.setValue(new File(sparkHome, "spark-executor").getCanonicalPath)
+ command.setValue(new File(sparkHome, "/sbin/spark-executor").getCanonicalPath)
} else {
// Grab everything to the first '.'. We'll use that and '*' to
// glob the directory "correctly".
val basename = uri.split('/').last.split('.').head
- command.setValue("cd %s*; ./spark-executor".format(basename))
+ command.setValue("cd %s*; ./sbin/spark-executor".format(basename))
command.addUris(CommandInfo.URI.newBuilder().setValue(uri))
}
val memory = Resource.newBuilder()
.setName("mem")
.setType(Value.Type.SCALAR)
- .setScalar(Value.Scalar.newBuilder().setValue(executorMemory).build())
+ .setScalar(Value.Scalar.newBuilder().setValue(sc.executorMemory).build())
.build()
ExecutorInfo.newBuilder()
.setExecutorId(ExecutorID.newBuilder().setValue(execId).build())
@@ -199,7 +198,7 @@ private[spark] class MesosSchedulerBackend(
def enoughMemory(o: Offer) = {
val mem = getResource(o.getResourcesList, "mem")
val slaveId = o.getSlaveId.getValue
- mem >= executorMemory || slaveIdsWithExecutors.contains(slaveId)
+ mem >= sc.executorMemory || slaveIdsWithExecutors.contains(slaveId)
}
for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) {
@@ -341,5 +340,5 @@ private[spark] class MesosSchedulerBackend(
}
// TODO: query Mesos for number of cores
- override def defaultParallelism() = System.getProperty("spark.default.parallelism", "8").toInt
+ override def defaultParallelism() = sc.conf.get("spark.default.parallelism", "8").toInt
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
new file mode 100644
index 0000000000..897d47a9ad
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala
@@ -0,0 +1,109 @@
+/*
+ * 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.local
+
+import java.nio.ByteBuffer
+
+import akka.actor.{Actor, ActorRef, Props}
+
+import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState}
+import org.apache.spark.TaskState.TaskState
+import org.apache.spark.executor.{Executor, ExecutorBackend}
+import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer}
+
+private case class ReviveOffers()
+
+private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer)
+
+private case class KillTask(taskId: Long)
+
+/**
+ * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on
+ * LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend
+ * and the ClusterScheduler.
+ */
+private[spark] class LocalActor(
+ scheduler: TaskSchedulerImpl,
+ executorBackend: LocalBackend,
+ private val totalCores: Int) extends Actor with Logging {
+
+ private var freeCores = totalCores
+
+ private val localExecutorId = "localhost"
+ private val localExecutorHostname = "localhost"
+
+ val executor = new Executor(
+ localExecutorId, localExecutorHostname, scheduler.conf.getAll, isLocal = true)
+
+ def receive = {
+ case ReviveOffers =>
+ reviveOffers()
+
+ case StatusUpdate(taskId, state, serializedData) =>
+ scheduler.statusUpdate(taskId, state, serializedData)
+ if (TaskState.isFinished(state)) {
+ freeCores += 1
+ reviveOffers()
+ }
+
+ case KillTask(taskId) =>
+ executor.killTask(taskId)
+ }
+
+ def reviveOffers() {
+ val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores))
+ for (task <- scheduler.resourceOffers(offers).flatten) {
+ freeCores -= 1
+ executor.launchTask(executorBackend, task.taskId, task.serializedTask)
+ }
+ }
+}
+
+/**
+ * LocalBackend is used when running a local version of Spark where the executor, backend, and
+ * master all run in the same JVM. It sits behind a ClusterScheduler and handles launching tasks
+ * on a single Executor (created by the LocalBackend) running locally.
+ */
+private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int)
+ extends SchedulerBackend with ExecutorBackend {
+
+ var localActor: ActorRef = null
+
+ override def start() {
+ localActor = SparkEnv.get.actorSystem.actorOf(
+ Props(new LocalActor(scheduler, this, totalCores)),
+ "LocalBackendActor")
+ }
+
+ override def stop() {
+ }
+
+ override def reviveOffers() {
+ localActor ! ReviveOffers
+ }
+
+ override def defaultParallelism() = totalCores
+
+ override def killTask(taskId: Long, executorId: String) {
+ localActor ! KillTask(taskId)
+ }
+
+ override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) {
+ localActor ! StatusUpdate(taskId, state, serializedData)
+ }
+}
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
deleted file mode 100644
index 2699f0b33e..0000000000
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
+++ /dev/null
@@ -1,219 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.scheduler.local
-
-import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicInteger
-
-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.{Executor, ExecutorBackend}
-import org.apache.spark.scheduler._
-import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
-
-
-/**
- * A FIFO or Fair TaskScheduler implementation that runs tasks locally in a thread pool. Optionally
- * the scheduler also allows each task to fail up to maxFailures times, which is useful for
- * testing fault recovery.
- */
-
-private[local]
-case class LocalReviveOffers()
-
-private[local]
-case class LocalStatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer)
-
-private[local]
-case class KillTask(taskId: Long)
-
-private[spark]
-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) =>
- if (TaskState.isFinished(state)) {
- freeCores += 1
- launchTask(localScheduler.resourceOffer(freeCores))
- }
-
- case KillTask(taskId) =>
- executor.killTask(taskId)
- }
-
- private def launchTask(tasks: Seq[TaskDescription]) {
- for (task <- tasks) {
- freeCores -= 1
- 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 {
-
- val env = SparkEnv.get
- val attemptId = new AtomicInteger
- var dagScheduler: DAGScheduler = null
-
- // Application dependencies (added through SparkContext) that we've fetched so far on this node.
- // Each map holds the master's timestamp for the version of that file or JAR we got.
- val currentFiles: HashMap[String, Long] = new HashMap[String, Long]()
- val currentJars: HashMap[String, Long] = new HashMap[String, Long]()
-
- var schedulableBuilder: SchedulableBuilder = null
- var rootPool: Pool = null
- val schedulingMode: SchedulingMode = SchedulingMode.withName(
- 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]]
-
- var localActor: ActorRef = null
-
- override def start() {
- // temporarily set rootPool name to empty
- rootPool = new Pool("", schedulingMode, 0, 0)
- schedulableBuilder = {
- schedulingMode match {
- case SchedulingMode.FIFO =>
- new FIFOSchedulableBuilder(rootPool)
- case SchedulingMode.FAIR =>
- new FairSchedulableBuilder(rootPool)
- }
- }
- schedulableBuilder.buildPools()
-
- localActor = env.actorSystem.actorOf(Props(new LocalActor(this, threads)), "Test")
- }
-
- override def setDAGScheduler(dagScheduler: DAGScheduler) {
- this.dagScheduler = dagScheduler
- }
-
- override def submitTasks(taskSet: TaskSet) {
- synchronized {
- val manager = new LocalTaskSetManager(this, taskSet)
- schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties)
- activeTaskSets(taskSet.id) = manager
- taskSetTaskIds(taskSet.id) = new HashSet[Long]()
- localActor ! LocalReviveOffers
- }
- }
-
- 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
- val tasks = new ArrayBuffer[TaskDescription](freeCores)
- val sortedTaskSetQueue = rootPool.getSortedTaskSetQueue()
- for (manager <- sortedTaskSetQueue) {
- logDebug("parentName:%s,name:%s,runningTasks:%s".format(
- manager.parent.name, manager.name, manager.runningTasks))
- }
-
- var launchTask = false
- for (manager <- sortedTaskSetQueue) {
- do {
- launchTask = false
- manager.resourceOffer(null, null, freeCpuCores, null) match {
- case Some(task) =>
- tasks += task
- taskIdToTaskSetId(task.taskId) = manager.taskSet.id
- taskSetTaskIds(manager.taskSet.id) += task.taskId
- freeCpuCores -= 1
- launchTask = true
- case None => {}
- }
- } while(launchTask)
- }
- return tasks
- }
- }
-
- def taskSetFinished(manager: TaskSetManager) {
- synchronized {
- activeTaskSets -= manager.taskSet.id
- manager.parent.removeSchedulable(manager)
- logInfo("Remove TaskSet %s from pool %s".format(manager.taskSet.id, manager.parent.name))
- taskIdToTaskSetId --= taskSetTaskIds(manager.taskSet.id)
- taskSetTaskIds -= manager.taskSet.id
- }
- }
-
- 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)
- }
- }
-
- 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
deleted file mode 100644
index 53bf78267e..0000000000
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.scheduler.local
-
-import java.nio.ByteBuffer
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-
-import org.apache.spark.{ExceptionFailure, Logging, SparkEnv, SparkException, Success, TaskState}
-import org.apache.spark.TaskState.TaskState
-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: Pool = null
- var weight: Int = 1
- var minShare: Int = 0
- var runningTasks: Int = 0
- var priority: Int = taskSet.priority
- var stageId: Int = taskSet.stageId
- var name: String = "TaskSet_" + taskSet.stageId.toString
-
- var failCount = new Array[Int](taskSet.tasks.size)
- val taskInfos = new HashMap[Long, TaskInfo]
- val numTasks = taskSet.tasks.size
- var numFinished = 0
- val env = SparkEnv.get
- val ser = env.closureSerializer.newInstance()
- val copiesRunning = new Array[Int](numTasks)
- val finished = new Array[Boolean](numTasks)
- val numFailures = new Array[Int](numTasks)
- val MAX_TASK_FAILURES = sched.maxFailures
-
- def increaseRunningTasks(taskNum: Int): Unit = {
- runningTasks += taskNum
- if (parent != null) {
- parent.increaseRunningTasks(taskNum)
- }
- }
-
- def decreaseRunningTasks(taskNum: Int): Unit = {
- runningTasks -= taskNum
- if (parent != null) {
- parent.decreaseRunningTasks(taskNum)
- }
- }
-
- override def addSchedulable(schedulable: Schedulable): Unit = {
- // nothing
- }
-
- override def removeSchedulable(schedulable: Schedulable): Unit = {
- // nothing
- }
-
- override def getSchedulableByName(name: String): Schedulable = {
- return null
- }
-
- override def executorLost(executorId: String, host: String): Unit = {
- // nothing
- }
-
- override def checkSpeculatableTasks() = true
-
- override def getSortedTaskSetQueue(): ArrayBuffer[TaskSetManager] = {
- var sortedTaskSetQueue = new ArrayBuffer[TaskSetManager]
- sortedTaskSetQueue += this
- return sortedTaskSetQueue
- }
-
- override def hasPendingTasks() = true
-
- def findTask(): Option[Int] = {
- for (i <- 0 to numTasks-1) {
- if (copiesRunning(i) == 0 && !finished(i)) {
- return Some(i)
- }
- }
- return None
- }
-
- override def resourceOffer(
- execId: String,
- host: String,
- availableCpus: Int,
- maxLocality: TaskLocality.TaskLocality)
- : Option[TaskDescription] =
- {
- SparkEnv.set(sched.env)
- logDebug("availableCpus:%d, numFinished:%d, numTasks:%d".format(
- availableCpus.toInt, numFinished, numTasks))
- if (availableCpus > 0 && numFinished < numTasks) {
- findTask() match {
- case Some(index) =>
- val taskId = sched.attemptId.getAndIncrement()
- val task = taskSet.tasks(index)
- val info = new TaskInfo(taskId, index, System.currentTimeMillis(), "local", "local:1",
- TaskLocality.NODE_LOCAL)
- taskInfos(taskId) = info
- // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here
- // we assume the task can be serialized without exceptions.
- val bytes = Task.serializeWithDependencies(
- task, sched.sc.addedFiles, sched.sc.addedJars, ser)
- logInfo("Size of task " + taskId + " is " + bytes.limit + " bytes")
- val taskName = "task %s:%d".format(taskSet.id, index)
- copiesRunning(index) += 1
- increaseRunningTasks(1)
- taskStarted(task, info)
- return Some(new TaskDescription(taskId, null, taskName, index, bytes))
- case None => {}
- }
- }
- return None
- }
-
- def taskStarted(task: Task[_], info: TaskInfo) {
- sched.dagScheduler.taskStarted(task, info)
- }
-
- def taskEnded(tid: Long, state: TaskState, serializedData: ByteBuffer) {
- val info = taskInfos(tid)
- val index = info.index
- val task = taskSet.tasks(index)
- info.markSuccessful()
- 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.dagScheduler.taskEnded(task, Success, result.value, result.accumUpdates, info,
- result.metrics)
- numFinished += 1
- decreaseRunningTasks(1)
- finished(index) = true
- if (numFinished == numTasks) {
- sched.taskSetFinished(this)
- }
- }
-
- def taskFailed(tid: Long, state: TaskState, serializedData: ByteBuffer) {
- val info = taskInfos(tid)
- val index = info.index
- val task = taskSet.tasks(index)
- info.markFailed()
- decreaseRunningTasks(1)
- val reason: ExceptionFailure = ser.deserialize[ExceptionFailure](
- serializedData, getClass.getClassLoader)
- sched.dagScheduler.taskEnded(task, reason, null, null, info, reason.metrics.getOrElse(null))
- if (!finished(index)) {
- copiesRunning(index) -= 1
- numFailures(index) += 1
- val locs = reason.stackTrace.map(loc => "\tat %s".format(loc.toString))
- logInfo("Loss was due to %s\n%s\n%s".format(
- reason.className, reason.description, locs.mkString("\n")))
- if (numFailures(index) > MAX_TASK_FAILURES) {
- val errorMessage = "Task %s:%d failed more than %d times; aborting job %s".format(
- taskSet.id, index, MAX_TASK_FAILURES, reason.description)
- decreaseRunningTasks(runningTasks)
- sched.dagScheduler.taskSetFailed(taskSet, errorMessage)
- // need to delete failed Taskset from schedule queue
- sched.taskSetFinished(this)
- }
- }
- }
-
- override def error(message: String) {
- sched.dagScheduler.taskSetFailed(taskSet, message)
- sched.taskSetFinished(this)
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index 4de81617b1..5d3d43623d 100644
--- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
@@ -21,6 +21,7 @@ import java.io._
import java.nio.ByteBuffer
import org.apache.spark.util.ByteBufferInputStream
+import org.apache.spark.SparkConf
private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream {
val objOut = new ObjectOutputStream(out)
@@ -77,6 +78,6 @@ private[spark] class JavaSerializerInstance extends SerializerInstance {
/**
* A Spark serializer that uses Java's built-in serialization.
*/
-class JavaSerializer extends Serializer {
+class JavaSerializer(conf: SparkConf) extends Serializer {
def newInstance(): SerializerInstance = new JavaSerializerInstance
}
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 e748c2275d..a24a3b04b8 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -25,18 +25,18 @@ import com.esotericsoftware.kryo.{KryoException, Kryo}
import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
import com.twitter.chill.{EmptyScalaKryoInstantiator, AllScalaRegistrar}
-import org.apache.spark.{SerializableWritable, Logging}
+import org.apache.spark._
import org.apache.spark.broadcast.HttpBroadcast
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage._
+import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock}
/**
* A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]].
*/
-class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging {
-
+class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging {
private val bufferSize = {
- System.getProperty("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
+ conf.get("spark.kryoserializer.buffer.mb", "2").toInt * 1024 * 1024
}
def newKryoOutput() = new KryoOutput(bufferSize)
@@ -48,7 +48,7 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
// Allow disabling Kryo reference tracking if user knows their object graphs don't have loops.
// Do this before we invoke the user registrator so the user registrator can override this.
- kryo.setReferences(System.getProperty("spark.kryo.referenceTracking", "true").toBoolean)
+ kryo.setReferences(conf.get("spark.kryo.referenceTracking", "true").toBoolean)
for (cls <- KryoSerializer.toRegister) kryo.register(cls)
@@ -58,13 +58,13 @@ class KryoSerializer extends org.apache.spark.serializer.Serializer with Logging
// Allow the user to register their own classes by setting spark.kryo.registrator
try {
- Option(System.getProperty("spark.kryo.registrator")).foreach { regCls =>
+ for (regCls <- conf.getOption("spark.kryo.registrator")) {
logDebug("Running user registrator: " + regCls)
val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]
reg.registerClasses(kryo)
}
} catch {
- case _: Exception => println("Failed to register spark.kryo.registrator")
+ case e: Exception => logError("Failed to run spark.kryo.registrator", e)
}
// Register Chill's classes; we do this after our ranges and the user's own classes to let
diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
index 160cca4d6c..9a5e3cb77e 100644
--- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala
@@ -29,6 +29,9 @@ import org.apache.spark.util.{NextIterator, ByteBufferInputStream}
* A serializer. Because some serialization libraries are not thread safe, this class is used to
* create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual serialization and are
* guaranteed to only be called from one thread at a time.
+ *
+ * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a
+ * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes precedence.
*/
trait Serializer {
def newInstance(): SerializerInstance
diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
index 2955986fec..36a37af4f8 100644
--- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala
@@ -18,6 +18,7 @@
package org.apache.spark.serializer
import java.util.concurrent.ConcurrentHashMap
+import org.apache.spark.SparkConf
/**
@@ -26,18 +27,19 @@ import java.util.concurrent.ConcurrentHashMap
* creating a new one.
*/
private[spark] class SerializerManager {
+ // TODO: Consider moving this into SparkConf itself to remove the global singleton.
private val serializers = new ConcurrentHashMap[String, Serializer]
private var _default: Serializer = _
def default = _default
- def setDefault(clsName: String): Serializer = {
- _default = get(clsName)
+ def setDefault(clsName: String, conf: SparkConf): Serializer = {
+ _default = get(clsName, conf)
_default
}
- def get(clsName: String): Serializer = {
+ def get(clsName: String, conf: SparkConf): Serializer = {
if (clsName == null) {
default
} else {
@@ -51,8 +53,19 @@ private[spark] class SerializerManager {
serializer = serializers.get(clsName)
if (serializer == null) {
val clsLoader = Thread.currentThread.getContextClassLoader
- serializer =
- Class.forName(clsName, true, clsLoader).newInstance().asInstanceOf[Serializer]
+ val cls = Class.forName(clsName, true, clsLoader)
+
+ // First try with the constructor that takes SparkConf. If we can't find one,
+ // use a no-arg constructor instead.
+ try {
+ val constructor = cls.getConstructor(classOf[SparkConf])
+ serializer = constructor.newInstance(conf).asInstanceOf[Serializer]
+ } catch {
+ case _: NoSuchMethodException =>
+ val constructor = cls.getConstructor()
+ serializer = constructor.newInstance().asInstanceOf[Serializer]
+ }
+
serializers.put(clsName, serializer)
}
serializer
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 e51c5b30a3..47478631a1 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala
@@ -312,7 +312,7 @@ object BlockFetcherIterator {
logDebug("Sending request for %d blocks (%s) from %s".format(
req.blocks.size, Utils.bytesToString(req.size), req.address.host))
val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort)
- val cpier = new ShuffleCopier
+ val cpier = new ShuffleCopier(blockManager.conf)
cpier.getBlocks(cmId, req.blocks, putResult)
logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host )
}
@@ -327,7 +327,7 @@ object BlockFetcherIterator {
fetchRequestsSync.put(request)
}
- copiers = startCopiers(System.getProperty("spark.shuffle.copier.threads", "6").toInt)
+ copiers = startCopiers(conf.get("spark.shuffle.copier.threads", "6").toInt)
logInfo("Started " + fetchRequestsSync.size + " remote gets in " +
Utils.getUsedTimeMs(startTime))
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 702aca8323..6d2cda97b0 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -24,13 +24,13 @@ import scala.collection.mutable.{HashMap, ArrayBuffer}
import scala.util.Random
import akka.actor.{ActorSystem, Cancellable, Props}
-import akka.dispatch.{Await, Future}
-import akka.util.Duration
-import akka.util.duration._
+import scala.concurrent.{Await, Future}
+import scala.concurrent.duration.Duration
+import scala.concurrent.duration._
import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream}
-import org.apache.spark.{Logging, SparkEnv, SparkException}
+import org.apache.spark.{SparkConf, Logging, SparkEnv, SparkException}
import org.apache.spark.io.CompressionCodec
import org.apache.spark.network._
import org.apache.spark.serializer.Serializer
@@ -43,12 +43,13 @@ private[spark] class BlockManager(
actorSystem: ActorSystem,
val master: BlockManagerMaster,
val defaultSerializer: Serializer,
- maxMemory: Long)
+ maxMemory: Long,
+ val conf: SparkConf)
extends Logging {
val shuffleBlockManager = new ShuffleBlockManager(this)
val diskBlockManager = new DiskBlockManager(shuffleBlockManager,
- System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
+ conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")))
private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo]
@@ -57,12 +58,12 @@ private[spark] class BlockManager(
// If we use Netty for shuffle, start a new Netty-based shuffle sender service.
private val nettyPort: Int = {
- val useNetty = System.getProperty("spark.shuffle.use.netty", "false").toBoolean
- val nettyPortConfig = System.getProperty("spark.shuffle.sender.port", "0").toInt
+ val useNetty = conf.get("spark.shuffle.use.netty", "false").toBoolean
+ val nettyPortConfig = conf.get("spark.shuffle.sender.port", "0").toInt
if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0
}
- val connectionManager = new ConnectionManager(0)
+ val connectionManager = new ConnectionManager(0, conf)
implicit val futureExecContext = connectionManager.futureExecContext
val blockManagerId = BlockManagerId(
@@ -71,18 +72,18 @@ private[spark] class BlockManager(
// Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory
// for receiving shuffle outputs)
val maxBytesInFlight =
- System.getProperty("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024
+ conf.get("spark.reducer.maxMbInFlight", "48").toLong * 1024 * 1024
// Whether to compress broadcast variables that are stored
- val compressBroadcast = System.getProperty("spark.broadcast.compress", "true").toBoolean
+ val compressBroadcast = conf.get("spark.broadcast.compress", "true").toBoolean
// Whether to compress shuffle output that are stored
- val compressShuffle = System.getProperty("spark.shuffle.compress", "true").toBoolean
+ val compressShuffle = conf.get("spark.shuffle.compress", "true").toBoolean
// Whether to compress RDD partitions that are stored serialized
- val compressRdds = System.getProperty("spark.rdd.compress", "false").toBoolean
+ val compressRdds = conf.get("spark.rdd.compress", "false").toBoolean
- val heartBeatFrequency = BlockManager.getHeartBeatFrequencyFromSystemProperties
+ val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf)
- val hostPort = Utils.localHostPort()
+ val hostPort = Utils.localHostPort(conf)
val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)),
name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next)
@@ -100,8 +101,11 @@ private[spark] class BlockManager(
var heartBeatTask: Cancellable = null
- private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks)
- private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks)
+ private val metadataCleaner = new MetadataCleaner(
+ MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf)
+ private val broadcastCleaner = new MetadataCleaner(
+ MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf)
+
initialize()
// The compression codec to use. Note that the "lazy" val is necessary because we want to delay
@@ -109,14 +113,14 @@ private[spark] class BlockManager(
// program could be using a user-defined codec in a third party jar, which is loaded in
// Executor.updateDependencies. When the BlockManager is initialized, user level jars hasn't been
// loaded yet.
- private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec()
+ private lazy val compressionCodec: CompressionCodec = CompressionCodec.createCodec(conf)
/**
* Construct a BlockManager with a memory limit set based on system properties.
*/
def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster,
- serializer: Serializer) = {
- this(execId, actorSystem, master, serializer, BlockManager.getMaxMemoryFromSystemProperties)
+ serializer: Serializer, conf: SparkConf) = {
+ this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf)
}
/**
@@ -126,7 +130,7 @@ private[spark] class BlockManager(
private def initialize() {
master.registerBlockManager(blockManagerId, maxMemory, slaveActor)
BlockManagerWorker.startBlockManagerWorker(this)
- if (!BlockManager.getDisableHeartBeatsForTesting) {
+ if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) {
heartBeat()
}
@@ -439,7 +443,7 @@ private[spark] class BlockManager(
: BlockFetcherIterator = {
val iter =
- if (System.getProperty("spark.shuffle.use.netty", "false").toBoolean) {
+ if (conf.get("spark.shuffle.use.netty", "false").toBoolean) {
new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer)
} else {
new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer)
@@ -465,7 +469,8 @@ private[spark] class BlockManager(
def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int)
: BlockObjectWriter = {
val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _)
- new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream)
+ val syncWrites = conf.get("spark.shuffle.sync", "false").toBoolean
+ new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites)
}
/**
@@ -856,19 +861,18 @@ private[spark] class BlockManager(
private[spark] object BlockManager extends Logging {
-
val ID_GENERATOR = new IdGenerator
- def getMaxMemoryFromSystemProperties: Long = {
- val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble
+ def getMaxMemory(conf: SparkConf): Long = {
+ val memoryFraction = conf.get("spark.storage.memoryFraction", "0.66").toDouble
(Runtime.getRuntime.maxMemory * memoryFraction).toLong
}
- def getHeartBeatFrequencyFromSystemProperties: Long =
- System.getProperty("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
+ def getHeartBeatFrequency(conf: SparkConf): Long =
+ conf.get("spark.storage.blockManagerTimeoutIntervalMs", "60000").toLong / 4
- def getDisableHeartBeatsForTesting: Boolean =
- System.getProperty("spark.test.disableBlockManagerHeartBeat", "false").toBoolean
+ def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean =
+ conf.get("spark.test.disableBlockManagerHeartBeat", "false").toBoolean
/**
* Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that
@@ -924,4 +928,3 @@ private[spark] object BlockManager extends Logging {
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 94038649b3..51a29ed8ef 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -17,23 +17,25 @@
package org.apache.spark.storage
-import akka.actor.ActorRef
-import akka.dispatch.{Await, Future}
+import scala.concurrent.{Await, Future}
+import scala.concurrent.ExecutionContext.Implicits.global
+
+import akka.actor._
import akka.pattern.ask
-import akka.util.Duration
-import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.{SparkConf, Logging, SparkException}
import org.apache.spark.storage.BlockManagerMessages._
+import org.apache.spark.util.AkkaUtils
+private[spark]
+class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging {
-private[spark] class BlockManagerMaster(var driverActor: ActorRef) extends Logging {
-
- val AKKA_RETRY_ATTEMPTS: Int = System.getProperty("spark.akka.num.retries", "3").toInt
- val AKKA_RETRY_INTERVAL_MS: Int = System.getProperty("spark.akka.retry.wait", "3000").toInt
+ val AKKA_RETRY_ATTEMPTS: Int = conf.get("spark.akka.num.retries", "3").toInt
+ val AKKA_RETRY_INTERVAL_MS: Int = conf.get("spark.akka.retry.wait", "3000").toInt
val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster"
- val timeout = Duration.create(System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
+ val timeout = AkkaUtils.askTimeout(conf)
/** Remove a dead executor from the driver actor. This is only called on the driver side. */
def removeExecutor(execId: String) {
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
index f8cf14b503..58452d9657 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
@@ -21,24 +21,22 @@ import java.util.{HashMap => JHashMap}
import scala.collection.mutable
import scala.collection.JavaConversions._
+import scala.concurrent.Future
+import scala.concurrent.duration._
import akka.actor.{Actor, ActorRef, Cancellable}
-import akka.dispatch.Future
import akka.pattern.ask
-import akka.util.Duration
-import akka.util.duration._
-import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.{SparkConf, Logging, SparkException}
import org.apache.spark.storage.BlockManagerMessages._
-import org.apache.spark.util.Utils
-
+import org.apache.spark.util.{AkkaUtils, Utils}
/**
* BlockManagerMasterActor is an actor on the master node to track statuses of
* all slaves' block managers.
*/
private[spark]
-class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
+class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging {
// Mapping from block manager id to the block manager's information.
private val blockManagerInfo =
@@ -50,21 +48,19 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
// Mapping from block id to the set of block managers that have the block.
private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]]
- val akkaTimeout = Duration.create(
- System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
-
- initLogging()
+ private val akkaTimeout = AkkaUtils.askTimeout(conf)
- val slaveTimeout = System.getProperty("spark.storage.blockManagerSlaveTimeoutMs",
- "" + (BlockManager.getHeartBeatFrequencyFromSystemProperties * 3)).toLong
+ val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs",
+ "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong
- val checkTimeoutInterval = System.getProperty("spark.storage.blockManagerTimeoutIntervalMs",
+ val checkTimeoutInterval = conf.get("spark.storage.blockManagerTimeoutIntervalMs",
"60000").toLong
var timeoutCheckingTask: Cancellable = null
override def preStart() {
- if (!BlockManager.getDisableHeartBeatsForTesting) {
+ if (!BlockManager.getDisableHeartBeatsForTesting(conf)) {
+ import context.dispatcher
timeoutCheckingTask = context.system.scheduler.schedule(
0.seconds, checkTimeoutInterval.milliseconds, self, ExpireDeadHosts)
}
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 0c66addf9d..21f003609b 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala
@@ -30,7 +30,6 @@ import org.apache.spark.util.Utils
* TODO: Use event model.
*/
private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
- initLogging()
blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive)
@@ -101,8 +100,6 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends
private[spark] object BlockManagerWorker extends Logging {
private var blockManagerWorker: BlockManagerWorker = null
- initLogging()
-
def startBlockManagerWorker(manager: BlockManager) {
blockManagerWorker = new BlockManagerWorker(manager)
}
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 6ce9127c74..a06f50a0ac 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala
@@ -37,8 +37,6 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM
def length = blockMessages.length
- initLogging()
-
def set(bufferMessage: BufferMessage) {
val startTime = System.currentTimeMillis
val newBlockMessages = new ArrayBuffer[BlockMessage]()
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 b4451fc7b8..61e63c60d5 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala
@@ -74,7 +74,8 @@ class DiskBlockObjectWriter(
file: File,
serializer: Serializer,
bufferSize: Int,
- compressStream: OutputStream => OutputStream)
+ compressStream: OutputStream => OutputStream,
+ syncWrites: Boolean)
extends BlockObjectWriter(blockId)
with Logging
{
@@ -97,8 +98,6 @@ class DiskBlockObjectWriter(
override def flush() = out.flush()
}
- private val syncWrites = System.getProperty("spark.shuffle.sync", "false").toBoolean
-
/** The file channel, used for repositioning / truncating the file. */
private var channel: FileChannel = null
private var bs: OutputStream = null
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
index fcd2e97982..55dcb3742c 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -38,7 +38,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD
extends PathResolver with Logging {
private val MAX_DIR_CREATION_ATTEMPTS: Int = 10
- private val subDirsPerLocalDir = System.getProperty("spark.diskStore.subDirectories", "64").toInt
+ private val subDirsPerLocalDir = shuffleManager.conf.get("spark.diskStore.subDirectories", "64").toInt
// Create one local directory for each path mentioned in spark.local.dir; then, inside this
// directory, create multiple subdirectories that we will hash files into, in order to avoid
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 2f1b049ce4..39dc7bb19a 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala
@@ -59,32 +59,40 @@ private[spark] trait ShuffleWriterGroup {
*/
private[spark]
class ShuffleBlockManager(blockManager: BlockManager) {
+ def conf = blockManager.conf
+
// Turning off shuffle file consolidation causes all shuffle Blocks to get their own file.
// TODO: Remove this once the shuffle file consolidation feature is stable.
val consolidateShuffleFiles =
- System.getProperty("spark.shuffle.consolidateFiles", "true").toBoolean
+ conf.get("spark.shuffle.consolidateFiles", "false").toBoolean
- private val bufferSize = System.getProperty("spark.shuffle.file.buffer.kb", "100").toInt * 1024
+ private val bufferSize = conf.get("spark.shuffle.file.buffer.kb", "100").toInt * 1024
/**
* Contains all the state related to a particular shuffle. This includes a pool of unused
* ShuffleFileGroups, as well as all ShuffleFileGroups that have been created for the shuffle.
*/
- private class ShuffleState() {
+ private class ShuffleState(val numBuckets: Int) {
val nextFileId = new AtomicInteger(0)
val unusedFileGroups = new ConcurrentLinkedQueue[ShuffleFileGroup]()
val allFileGroups = new ConcurrentLinkedQueue[ShuffleFileGroup]()
+
+ /**
+ * The mapIds of all map tasks completed on this Executor for this shuffle.
+ * NB: This is only populated if consolidateShuffleFiles is FALSE. We don't need it otherwise.
+ */
+ val completedMapTasks = new ConcurrentLinkedQueue[Int]()
}
type ShuffleId = Int
private val shuffleStates = new TimeStampedHashMap[ShuffleId, ShuffleState]
- private
- val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup)
+ private val metadataCleaner =
+ new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf)
def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = {
new ShuffleWriterGroup {
- shuffleStates.putIfAbsent(shuffleId, new ShuffleState())
+ shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets))
private val shuffleState = shuffleStates(shuffleId)
private var fileGroup: ShuffleFileGroup = null
@@ -109,6 +117,8 @@ class ShuffleBlockManager(blockManager: BlockManager) {
fileGroup.recordMapOutput(mapId, offsets)
}
recycleFileGroup(fileGroup)
+ } else {
+ shuffleState.completedMapTasks.add(mapId)
}
}
@@ -154,7 +164,18 @@ class ShuffleBlockManager(blockManager: BlockManager) {
}
private def cleanup(cleanupTime: Long) {
- shuffleStates.clearOldValues(cleanupTime)
+ shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => {
+ if (consolidateShuffleFiles) {
+ for (fileGroup <- state.allFileGroups; file <- fileGroup.files) {
+ file.delete()
+ }
+ } else {
+ for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) {
+ val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId)
+ blockManager.diskBlockManager.getFile(blockId).delete()
+ }
+ }
+ })
}
}
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 632ff047d1..b5596dffd3 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
@@ -101,7 +101,7 @@ class StorageLevel private(
var result = ""
result += (if (useDisk) "Disk " else "")
result += (if (useMemory) "Memory " else "")
- result += (if (deserialized) "Deserialized " else "Serialized")
+ result += (if (deserialized) "Deserialized " else "Serialized ")
result += "%sx Replicated".format(replication)
result
}
diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
index 1e4db4f66b..40734aab49 100644
--- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.storage
import java.util.concurrent.atomic.AtomicLong
@@ -39,7 +56,7 @@ object StoragePerfTester {
def writeOutputBytes(mapId: Int, total: AtomicLong) = {
val shuffle = blockManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits,
- new KryoSerializer())
+ new KryoSerializer(sc.conf))
val writers = shuffle.writers
for (i <- 1 to recordsPerMap) {
writers(i % numOutputSplits).write(writeData)
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 860e680576..729ba2c550 100644
--- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
@@ -22,6 +22,7 @@ import akka.actor._
import java.util.concurrent.ArrayBlockingQueue
import util.Random
import org.apache.spark.serializer.KryoSerializer
+import org.apache.spark.{SparkConf, SparkContext}
/**
* This class tests the BlockManager and MemoryStore for thread safety and
@@ -91,11 +92,12 @@ private[spark] object ThreadingTest {
def main(args: Array[String]) {
System.setProperty("spark.kryoserializer.buffer.mb", "1")
val actorSystem = ActorSystem("test")
- val serializer = new KryoSerializer
+ val conf = new SparkConf()
+ val serializer = new KryoSerializer(conf)
val blockManagerMaster = new BlockManagerMaster(
- actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
+ actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf)
val blockManager = new BlockManager(
- "<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024)
+ "<driver>", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf)
val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i))
val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue))
producers.foreach(_.start)
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 f1d86c0221..50dfdbdf5a 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.
@@ -32,7 +32,7 @@ import org.apache.spark.util.Utils
/** Top level user interface for Spark */
private[spark] class SparkUI(sc: SparkContext) extends Logging {
val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName())
- val port = Option(System.getProperty("spark.ui.port")).getOrElse(SparkUI.DEFAULT_PORT).toInt
+ val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt
var boundPort: Option[Int] = None
var server: Option[Server] = None
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 fcd1b518d0..6ba15187d9 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
@@ -19,7 +19,7 @@ package org.apache.spark.ui
import scala.util.Random
-import org.apache.spark.SparkContext
+import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.SparkContext._
import org.apache.spark.scheduler.SchedulingMode
@@ -27,25 +27,26 @@ import org.apache.spark.scheduler.SchedulingMode
/**
* Continuously generates jobs that expose various features of the WebUI (internal testing tool).
*
- * Usage: ./run spark.ui.UIWorkloadGenerator [master]
+ * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]
*/
private[spark] object UIWorkloadGenerator {
+
val NUM_PARTITIONS = 100
val INTER_JOB_WAIT_MS = 5000
def main(args: Array[String]) {
if (args.length < 2) {
- println("usage: ./spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
+ println("usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]")
System.exit(1)
}
- val master = args(0)
- val schedulingMode = SchedulingMode.withName(args(1))
- val appName = "Spark UI Tester"
+ val conf = new SparkConf().setMaster(args(0)).setAppName("Spark UI tester")
+
+ val schedulingMode = SchedulingMode.withName(args(1))
if (schedulingMode == SchedulingMode.FAIR) {
- System.setProperty("spark.scheduler.mode", "FAIR")
+ conf.set("spark.scheduler.mode", "FAIR")
}
- val sc = new SparkContext(master, appName)
+ val sc = new SparkContext(conf)
def setProperties(s: String) = {
if(schedulingMode == SchedulingMode.FAIR) {
@@ -55,11 +56,11 @@ private[spark] object UIWorkloadGenerator {
}
val baseData = sc.makeRDD(1 to NUM_PARTITIONS * 10, NUM_PARTITIONS)
- def nextFloat() = (new Random()).nextFloat()
+ def nextFloat() = new Random().nextFloat()
val jobs = Seq[(String, () => Long)](
("Count", baseData.count),
- ("Cache and Count", baseData.map(x => x).cache.count),
+ ("Cache and Count", baseData.map(x => x).cache().count),
("Single Shuffle", baseData.map(x => (x % 10, x)).reduceByKey(_ + _).count),
("Entirely failed phase", baseData.map(x => throw new Exception).count),
("Partially failed phase", {
diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
index c5bf2acc9e..88f41be8d3 100644
--- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
@@ -48,12 +48,15 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
def jvmTable =
UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true)
- val properties = System.getProperties.iterator.toSeq
- val classPathProperty = properties.find { case (k, v) =>
- k.contains("java.class.path")
+ val sparkProperties = sc.conf.getAll.sorted
+
+ val systemProperties = System.getProperties.iterator.toSeq
+ val classPathProperty = systemProperties.find { case (k, v) =>
+ k == "java.class.path"
}.getOrElse(("", ""))
- val sparkProperties = properties.filter(_._1.startsWith("spark")).sorted
- val otherProperties = properties.diff(sparkProperties :+ classPathProperty).sorted
+ val otherProperties = systemProperties.filter { case (k, v) =>
+ k != "java.class.path" && !k.startsWith("spark.")
+ }.sorted
val propertyHeaders = Seq("Name", "Value")
def propertyRow(kv: (String, String)) = <tr><td>{kv._1}</td><td>{kv._2}</td></tr>
@@ -63,7 +66,7 @@ private[spark] class EnvironmentUI(sc: SparkContext) {
UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true)
val classPathEntries = classPathProperty._2
- .split(System.getProperty("path.separator", ":"))
+ .split(sc.conf.get("path.separator", ":"))
.filterNot(e => e.isEmpty)
.map(e => (e, "System Classpath"))
val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")}
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 e596690bc3..a31a7e1d58 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
@@ -56,7 +56,8 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_+_)
val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used",
- "Active tasks", "Failed tasks", "Complete tasks", "Total tasks")
+ "Active tasks", "Failed tasks", "Complete tasks", "Total tasks", "Task Time", "Shuffle Read",
+ "Shuffle Write")
def execRow(kv: Seq[String]) = {
<tr>
@@ -73,6 +74,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
<td>{kv(7)}</td>
<td>{kv(8)}</td>
<td>{kv(9)}</td>
+ <td>{Utils.msDurationToString(kv(10).toLong)}</td>
+ <td>{Utils.bytesToString(kv(11).toLong)}</td>
+ <td>{Utils.bytesToString(kv(12).toLong)}</td>
</tr>
}
@@ -111,6 +115,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0)
val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0)
val totalTasks = activeTasks + failedTasks + completedTasks
+ val totalDuration = listener.executorToDuration.getOrElse(execId, 0)
+ val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0)
+ val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0)
Seq(
execId,
@@ -122,7 +129,10 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
activeTasks.toString,
failedTasks.toString,
completedTasks.toString,
- totalTasks.toString
+ totalTasks.toString,
+ totalDuration.toString,
+ totalShuffleRead.toString,
+ totalShuffleWrite.toString
)
}
@@ -130,6 +140,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]()
val executorToTasksComplete = HashMap[String, Int]()
val executorToTasksFailed = HashMap[String, Int]()
+ val executorToDuration = HashMap[String, Long]()
+ val executorToShuffleRead = HashMap[String, Long]()
+ val executorToShuffleWrite = HashMap[String, Long]()
override def onTaskStart(taskStart: SparkListenerTaskStart) {
val eid = taskStart.taskInfo.executorId
@@ -140,6 +153,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
val eid = taskEnd.taskInfo.executorId
val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]())
+ val newDuration = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration
+ executorToDuration.put(eid, newDuration)
+
activeTasks -= taskEnd.taskInfo
val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) =
taskEnd.reason match {
@@ -150,6 +166,17 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1
(None, Option(taskEnd.taskMetrics))
}
+
+ // update shuffle read/write
+ if (null != taskEnd.taskMetrics) {
+ taskEnd.taskMetrics.shuffleReadMetrics.foreach(shuffleRead =>
+ executorToShuffleRead.put(eid, executorToShuffleRead.getOrElse(eid, 0L) +
+ shuffleRead.remoteBytesRead))
+
+ taskEnd.taskMetrics.shuffleWriteMetrics.foreach(shuffleWrite =>
+ executorToShuffleWrite.put(eid, executorToShuffleWrite.getOrElse(eid, 0L) +
+ shuffleWrite.shuffleBytesWritten))
+ }
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala
new file mode 100644
index 0000000000..3c53e88380
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala
@@ -0,0 +1,27 @@
+/*
+ * 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.ui.jobs
+
+/** class for reporting aggregated metrics for each executors in stageUI */
+private[spark] class ExecutorSummary {
+ var taskTime : Long = 0
+ var failedTasks : Int = 0
+ var succeededTasks : Int = 0
+ var shuffleRead : Long = 0
+ var shuffleWrite : Long = 0
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala
new file mode 100644
index 0000000000..0dd876480a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.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.ui.jobs
+
+import scala.xml.Node
+
+import org.apache.spark.scheduler.SchedulingMode
+import org.apache.spark.util.Utils
+import scala.collection.mutable
+
+/** Page showing executor summary */
+private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) {
+
+ val listener = parent.listener
+ val dateFmt = parent.dateFmt
+ val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR
+
+ def toNodeSeq(): Seq[Node] = {
+ listener.synchronized {
+ executorTable()
+ }
+ }
+
+ /** Special table which merges two header cells. */
+ private def executorTable[T](): Seq[Node] = {
+ <table class="table table-bordered table-striped table-condensed sortable">
+ <thead>
+ <th>Executor ID</th>
+ <th>Address</th>
+ <th>Task Time</th>
+ <th>Total Tasks</th>
+ <th>Failed Tasks</th>
+ <th>Succeeded Tasks</th>
+ <th>Shuffle Read</th>
+ <th>Shuffle Write</th>
+ </thead>
+ <tbody>
+ {createExecutorTable()}
+ </tbody>
+ </table>
+ }
+
+ private def createExecutorTable() : Seq[Node] = {
+ // make a executor-id -> address map
+ val executorIdToAddress = mutable.HashMap[String, String]()
+ val storageStatusList = parent.sc.getExecutorStorageStatus
+ for (statusId <- 0 until storageStatusList.size) {
+ val blockManagerId = parent.sc.getExecutorStorageStatus(statusId).blockManagerId
+ val address = blockManagerId.hostPort
+ val executorId = blockManagerId.executorId
+ executorIdToAddress.put(executorId, address)
+ }
+
+ val executorIdToSummary = listener.stageIdToExecutorSummaries.get(stageId)
+ executorIdToSummary match {
+ case Some(x) => {
+ x.toSeq.sortBy(_._1).map{
+ case (k,v) => {
+ <tr>
+ <td>{k}</td>
+ <td>{executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}</td>
+ <td>{parent.formatDuration(v.taskTime)}</td>
+ <td>{v.failedTasks + v.succeededTasks}</td>
+ <td>{v.failedTasks}</td>
+ <td>{v.succeededTasks}</td>
+ <td>{Utils.bytesToString(v.shuffleRead)}</td>
+ <td>{Utils.bytesToString(v.shuffleWrite)}</td>
+ </tr>
+ }
+ }
+ }
+ case _ => { Seq[Node]() }
+ }
+ }
+}
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 6b854740d6..b7b87250b9 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
@@ -33,7 +33,7 @@ import org.apache.spark.scheduler._
*/
private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener {
// How many stages to remember
- val RETAINED_STAGES = System.getProperty("spark.ui.retained_stages", "1000").toInt
+ val RETAINED_STAGES = sc.conf.get("spark.ui.retained_stages", "1000").toInt
val DEFAULT_POOL_NAME = "default"
val stageIdToPool = new HashMap[Int, String]()
@@ -57,10 +57,11 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
val stageIdToTasksFailed = HashMap[Int, Int]()
val stageIdToTaskInfos =
HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]()
+ val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]()
override def onJobStart(jobStart: SparkListenerJobStart) {}
- override def onStageCompleted(stageCompleted: StageCompleted) = synchronized {
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized {
val stage = stageCompleted.stage
poolToActiveStages(stageIdToPool(stage.stageId)) -= stage
activeStages -= stage
@@ -105,7 +106,7 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]())
stages += stage
}
-
+
override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized {
val sid = taskStart.task.stageId
val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())
@@ -124,8 +125,38 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList
override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized {
val sid = taskEnd.task.stageId
+
+ // create executor summary map if necessary
+ val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid,
+ op = new HashMap[String, ExecutorSummary]())
+ executorSummaryMap.getOrElseUpdate(key = taskEnd.taskInfo.executorId,
+ op = new ExecutorSummary())
+
+ val executorSummary = executorSummaryMap.get(taskEnd.taskInfo.executorId)
+ executorSummary match {
+ case Some(y) => {
+ // first update failed-task, succeed-task
+ taskEnd.reason match {
+ case Success =>
+ y.succeededTasks += 1
+ case _ =>
+ y.failedTasks += 1
+ }
+
+ // update duration
+ y.taskTime += taskEnd.taskInfo.duration
+
+ Option(taskEnd.taskMetrics).foreach { taskMetrics =>
+ taskMetrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead }
+ taskMetrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten }
+ }
+ }
+ case _ => {}
+ }
+
val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]())
tasksActive -= taskEnd.taskInfo
+
val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) =
taskEnd.reason match {
case e: ExceptionFailure =>
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 e7eab374ad..c1ee2f3d00 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
@@ -17,7 +17,7 @@
package org.apache.spark.ui.jobs
-import akka.util.Duration
+import scala.concurrent.duration._
import java.text.SimpleDateFormat
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 fbd822867f..8dcfeacb60 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -60,11 +60,13 @@ private[spark] class StagePage(parent: JobProgressUI) {
var activeTime = 0L
listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now))
+ val finishedTasks = listener.stageIdToTaskInfos(stageId).filter(_._1.finished)
+
val summary =
<div>
<ul class="unstyled">
<li>
- <strong>CPU time: </strong>
+ <strong>Total task time across all tasks: </strong>
{parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
</li>
{if (hasShuffleRead)
@@ -84,7 +86,7 @@ private[spark] class StagePage(parent: JobProgressUI) {
val taskHeaders: Seq[String] =
Seq("Task Index", "Task ID", "Status", "Locality Level", "Executor", "Launch Time") ++
- Seq("Duration", "GC Time") ++
+ Seq("Duration", "GC Time", "Result Ser Time") ++
{if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++
{if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++
Seq("Errors")
@@ -99,11 +101,43 @@ private[spark] class StagePage(parent: JobProgressUI) {
None
}
else {
+ val serializationTimes = validTasks.map{case (info, metrics, exception) =>
+ metrics.get.resultSerializationTime.toDouble}
+ val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes).get.getQuantiles().map(
+ ms => parent.formatDuration(ms.toLong))
+
val serviceTimes = validTasks.map{case (info, metrics, exception) =>
metrics.get.executorRunTime.toDouble}
val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map(
ms => parent.formatDuration(ms.toLong))
+ val gettingResultTimes = validTasks.map{case (info, metrics, exception) =>
+ if (info.gettingResultTime > 0) {
+ (info.finishTime - info.gettingResultTime).toDouble
+ } else {
+ 0.0
+ }
+ }
+ val gettingResultQuantiles = ("Time spent fetching task results" +:
+ Distribution(gettingResultTimes).get.getQuantiles().map(
+ millis => parent.formatDuration(millis.toLong)))
+ // The scheduler delay includes the network delay to send the task to the worker
+ // machine and to send back the result (but not the time to fetch the task result,
+ // if it needed to be fetched from the block manager on the worker).
+ val schedulerDelays = validTasks.map{case (info, metrics, exception) =>
+ val totalExecutionTime = {
+ if (info.gettingResultTime > 0) {
+ (info.gettingResultTime - info.launchTime).toDouble
+ } else {
+ (info.finishTime - info.launchTime).toDouble
+ }
+ }
+ totalExecutionTime - metrics.get.executorRunTime
+ }
+ val schedulerDelayQuantiles = ("Scheduler delay" +:
+ Distribution(schedulerDelays).get.getQuantiles().map(
+ millis => parent.formatDuration(millis.toLong)))
+
def getQuantileCols(data: Seq[Double]) =
Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong))
@@ -119,7 +153,11 @@ private[spark] class StagePage(parent: JobProgressUI) {
}
val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes)
- val listings: Seq[Seq[String]] = Seq(serviceQuantiles,
+ val listings: Seq[Seq[String]] = Seq(
+ serializationQuantiles,
+ serviceQuantiles,
+ gettingResultQuantiles,
+ schedulerDelayQuantiles,
if (hasShuffleRead) shuffleReadQuantiles else Nil,
if (hasShuffleWrite) shuffleWriteQuantiles else Nil)
@@ -128,11 +166,12 @@ private[spark] class StagePage(parent: JobProgressUI) {
def quantileRow(data: Seq[String]): Seq[Node] = <tr> {data.map(d => <td>{d}</td>)} </tr>
Some(listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true))
}
-
+ val executorTable = new ExecutorTable(parent, stageId)
val content =
summary ++
<h4>Summary Metrics for {numCompleted} Completed Tasks</h4> ++
<div>{summaryTable.getOrElse("No tasks have reported metrics yet.")}</div> ++
+ <h4>Aggregated Metrics by Executors</h4> ++ executorTable.toNodeSeq() ++
<h4>Tasks</h4> ++ taskTable
headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages)
@@ -151,22 +190,20 @@ private[spark] class StagePage(parent: JobProgressUI) {
val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration)
else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("")
val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L)
+ val serializationTime = metrics.map(m => m.resultSerializationTime).getOrElse(0L)
- var shuffleReadSortable: String = ""
- var shuffleReadReadable: String = ""
- if (shuffleRead) {
- shuffleReadSortable = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead}.toString()
- shuffleReadReadable = metrics.flatMap{m => m.shuffleReadMetrics}.map{s =>
- Utils.bytesToString(s.remoteBytesRead)}.getOrElse("")
- }
+ val maybeShuffleRead = metrics.flatMap{m => m.shuffleReadMetrics}.map{s => s.remoteBytesRead}
+ val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("")
+ val shuffleReadReadable = maybeShuffleRead.map{Utils.bytesToString(_)}.getOrElse("")
- var shuffleWriteSortable: String = ""
- var shuffleWriteReadable: String = ""
- if (shuffleWrite) {
- shuffleWriteSortable = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten}.toString()
- shuffleWriteReadable = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s =>
- Utils.bytesToString(s.shuffleBytesWritten)}.getOrElse("")
- }
+ val maybeShuffleWrite = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleBytesWritten}
+ val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("")
+ val shuffleWriteReadable = maybeShuffleWrite.map{Utils.bytesToString(_)}.getOrElse("")
+
+ val maybeWriteTime = metrics.flatMap{m => m.shuffleWriteMetrics}.map{s => s.shuffleWriteTime}
+ val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("")
+ val writeTimeReadable = maybeWriteTime.map{ t => t / (1000 * 1000)}.map{ ms =>
+ if (ms == 0) "" else parent.formatDuration(ms)}.getOrElse("")
<tr>
<td>{info.index}</td>
@@ -181,14 +218,17 @@ private[spark] class StagePage(parent: JobProgressUI) {
<td sorttable_customkey={gcTime.toString}>
{if (gcTime > 0) parent.formatDuration(gcTime) else ""}
</td>
+ <td sorttable_customkey={serializationTime.toString}>
+ {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""}
+ </td>
{if (shuffleRead) {
<td sorttable_customkey={shuffleReadSortable}>
{shuffleReadReadable}
</td>
}}
{if (shuffleWrite) {
- <td>{metrics.flatMap{m => m.shuffleWriteMetrics}.map{s =>
- parent.formatDuration(s.shuffleWriteTime / (1000 * 1000))}.getOrElse("")}
+ <td sorttable_customkey={writeTimeSortable}>
+ {writeTimeReadable}
</td>
<td sorttable_customkey={shuffleWriteSortable}>
{shuffleWriteReadable}
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 9ad6de3c6d..463d85dfd5 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
@@ -48,7 +48,7 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr
{if (isFairScheduler) {<th>Pool Name</th>} else {}}
<th>Description</th>
<th>Submitted</th>
- <th>Duration</th>
+ <th>Task Time</th>
<th>Tasks: Succeeded/Total</th>
<th>Shuffle Read</th>
<th>Shuffle Write</th>
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
index 1d633d374a..39f422dd6b 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
@@ -17,7 +17,7 @@
package org.apache.spark.ui.storage
-import akka.util.Duration
+import scala.concurrent.duration._
import javax.servlet.http.HttpServletRequest
@@ -28,9 +28,6 @@ import org.apache.spark.ui.JettyUtils._
/** Web UI showing storage status of all RDD's in the given SparkContext. */
private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging {
- implicit val timeout = Duration.create(
- System.getProperty("spark.akka.askTimeout", "10").toLong, "seconds")
-
val indexPage = new IndexPage(this)
val rddPage = new RDDPage(this)
diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index d4c5065c3f..3f009a8998 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -17,11 +17,14 @@
package org.apache.spark.util
-import akka.actor.{ActorSystem, ExtendedActorSystem}
+import scala.collection.JavaConversions.mapAsJavaMap
+import scala.concurrent.duration.{Duration, FiniteDuration}
+
+import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem}
import com.typesafe.config.ConfigFactory
-import akka.util.duration._
-import akka.remote.RemoteActorRefProvider
+import org.apache.log4j.{Level, Logger}
+import org.apache.spark.SparkConf
/**
* Various utility classes for working with Akka.
@@ -34,39 +37,77 @@ private[spark] object AkkaUtils {
*
* Note: the `name` parameter is important, as even if a client sends a message to right
* host + port, if the system name is incorrect, Akka will drop the message.
+ *
+ * If indestructible is set to true, the Actor System will continue running in the event
+ * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]].
*/
- def createActorSystem(name: String, host: String, port: Int): (ActorSystem, Int) = {
- val akkaThreads = System.getProperty("spark.akka.threads", "4").toInt
- val akkaBatchSize = System.getProperty("spark.akka.batchSize", "15").toInt
- val akkaTimeout = System.getProperty("spark.akka.timeout", "60").toInt
- val akkaFrameSize = System.getProperty("spark.akka.frameSize", "10").toInt
- val lifecycleEvents = if (System.getProperty("spark.akka.logLifecycleEvents", "false").toBoolean) "on" else "off"
- // 10 seconds is the default akka timeout, but in a cluster, we need higher by default.
- val akkaWriteTimeout = System.getProperty("spark.akka.writeTimeout", "30").toInt
-
- val akkaConf = ConfigFactory.parseString("""
- akka.daemonic = on
- akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
- akka.stdout-loglevel = "ERROR"
- akka.actor.provider = "akka.remote.RemoteActorRefProvider"
- akka.remote.transport = "akka.remote.netty.NettyRemoteTransport"
- akka.remote.netty.hostname = "%s"
- akka.remote.netty.port = %d
- akka.remote.netty.connection-timeout = %ds
- akka.remote.netty.message-frame-size = %d MiB
- akka.remote.netty.execution-pool-size = %d
- akka.actor.default-dispatcher.throughput = %d
- akka.remote.log-remote-lifecycle-events = %s
- akka.remote.netty.write-timeout = %ds
- """.format(host, port, akkaTimeout, akkaFrameSize, akkaThreads, akkaBatchSize,
- lifecycleEvents, akkaWriteTimeout))
-
- val actorSystem = ActorSystem(name, akkaConf)
-
- // Figure out the port number we bound to, in case port was passed as 0. This is a bit of a
- // hack because Akka doesn't let you figure out the port through the public API yet.
+ def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false,
+ conf: SparkConf): (ActorSystem, Int) = {
+
+ val akkaThreads = conf.get("spark.akka.threads", "4").toInt
+ val akkaBatchSize = conf.get("spark.akka.batchSize", "15").toInt
+
+ val akkaTimeout = conf.get("spark.akka.timeout", "100").toInt
+
+ val akkaFrameSize = conf.get("spark.akka.frameSize", "10").toInt
+ val akkaLogLifecycleEvents = conf.get("spark.akka.logLifecycleEvents", "false").toBoolean
+ val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off"
+ if (!akkaLogLifecycleEvents) {
+ // As a workaround for Akka issue #3787, we coerce the "EndpointWriter" log to be silent.
+ // See: https://www.assembla.com/spaces/akka/tickets/3787#/
+ Option(Logger.getLogger("akka.remote.EndpointWriter")).map(l => l.setLevel(Level.FATAL))
+ }
+
+ val logAkkaConfig = if (conf.get("spark.akka.logAkkaConfig", "false").toBoolean) "on" else "off"
+
+ val akkaHeartBeatPauses = conf.get("spark.akka.heartbeat.pauses", "600").toInt
+ val akkaFailureDetector =
+ conf.get("spark.akka.failure-detector.threshold", "300.0").toDouble
+ val akkaHeartBeatInterval = conf.get("spark.akka.heartbeat.interval", "1000").toInt
+
+ val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback(
+ ConfigFactory.parseString(
+ s"""
+ |akka.daemonic = on
+ |akka.loggers = [""akka.event.slf4j.Slf4jLogger""]
+ |akka.stdout-loglevel = "ERROR"
+ |akka.jvm-exit-on-fatal-error = off
+ |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s
+ |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s
+ |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector
+ |akka.actor.provider = "akka.remote.RemoteActorRefProvider"
+ |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport"
+ |akka.remote.netty.tcp.hostname = "$host"
+ |akka.remote.netty.tcp.port = $port
+ |akka.remote.netty.tcp.tcp-nodelay = on
+ |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s
+ |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB
+ |akka.remote.netty.tcp.execution-pool-size = $akkaThreads
+ |akka.actor.default-dispatcher.throughput = $akkaBatchSize
+ |akka.log-config-on-start = $logAkkaConfig
+ |akka.remote.log-remote-lifecycle-events = $lifecycleEvents
+ |akka.log-dead-letters = $lifecycleEvents
+ |akka.log-dead-letters-during-shutdown = $lifecycleEvents
+ """.stripMargin))
+
+ val actorSystem = if (indestructible) {
+ IndestructibleActorSystem(name, akkaConf)
+ } else {
+ ActorSystem(name, akkaConf)
+ }
+
val provider = actorSystem.asInstanceOf[ExtendedActorSystem].provider
- val boundPort = provider.asInstanceOf[RemoteActorRefProvider].transport.address.port.get
- return (actorSystem, boundPort)
+ val boundPort = provider.getDefaultAddress.port.get
+ (actorSystem, boundPort)
+ }
+
+ /** Returns the default Spark timeout to use for Akka ask operations. */
+ def askTimeout(conf: SparkConf): FiniteDuration = {
+ Duration.create(conf.get("spark.akka.askTimeout", "30").toLong, "seconds")
+ }
+
+ /** Returns the default Spark timeout to use for Akka remote actor lookup. */
+ def lookupTimeout(conf: SparkConf): FiniteDuration = {
+ Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds")
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala
index 0b51c23f7b..a38329df03 100644
--- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala
+++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala
@@ -34,6 +34,8 @@ class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A])
override def iterator: Iterator[A] = underlying.iterator.asScala
+ override def size: Int = underlying.size
+
override def ++=(xs: TraversableOnce[A]): this.type = {
xs.foreach { this += _ }
this
diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala
new file mode 100644
index 0000000000..bf71882ef7
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+// Must be in akka.actor package as ActorSystemImpl is protected[akka].
+package akka.actor
+
+import scala.util.control.{ControlThrowable, NonFatal}
+
+import com.typesafe.config.Config
+
+/**
+ * An [[akka.actor.ActorSystem]] which refuses to shut down in the event of a fatal exception.
+ * This is necessary as Spark Executors are allowed to recover from fatal exceptions
+ * (see [[org.apache.spark.executor.Executor]]).
+ */
+object IndestructibleActorSystem {
+ def apply(name: String, config: Config): ActorSystem =
+ apply(name, config, ActorSystem.findClassLoader())
+
+ def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem =
+ new IndestructibleActorSystemImpl(name, config, classLoader).start()
+}
+
+private[akka] class IndestructibleActorSystemImpl(
+ override val name: String,
+ applicationConfig: Config,
+ classLoader: ClassLoader)
+ extends ActorSystemImpl(name, applicationConfig, classLoader) {
+
+ protected override def uncaughtExceptionHandler: Thread.UncaughtExceptionHandler = {
+ val fallbackHandler = super.uncaughtExceptionHandler
+
+ new Thread.UncaughtExceptionHandler() {
+ def uncaughtException(thread: Thread, cause: Throwable): Unit = {
+ if (isFatalError(cause) && !settings.JvmExitOnFatalError) {
+ log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " +
+ "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name)
+ //shutdown() //TODO make it configurable
+ } else {
+ fallbackHandler.uncaughtException(thread, cause)
+ }
+ }
+ }
+ }
+
+ def isFatalError(e: Throwable): Boolean = {
+ e match {
+ case NonFatal(_) | _: InterruptedException | _: NotImplementedError | _: ControlThrowable =>
+ false
+ case _ =>
+ true
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
index 67a7f87a5c..aa7f52cafb 100644
--- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
@@ -18,16 +18,21 @@
package org.apache.spark.util
import java.util.{TimerTask, Timer}
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, SparkContext, Logging}
/**
* Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries)
*/
-class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, cleanupFunc: (Long) => Unit) extends Logging {
+class MetadataCleaner(
+ cleanerType: MetadataCleanerType.MetadataCleanerType,
+ cleanupFunc: (Long) => Unit,
+ conf: SparkConf)
+ extends Logging
+{
val name = cleanerType.toString
- private val delaySeconds = MetadataCleaner.getDelaySeconds
+ private val delaySeconds = MetadataCleaner.getDelaySeconds(conf, cleanerType)
private val periodSeconds = math.max(10, delaySeconds / 10)
private val timer = new Timer(name + " cleanup timer", true)
@@ -55,8 +60,7 @@ class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, clea
}
}
-object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext", "HttpBroadcast", "DagScheduler", "ResultTask",
- "ShuffleMapTask", "BlockManager", "DiskBlockManager", "BroadcastVars") {
+object MetadataCleanerType extends Enumeration {
val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK,
SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value
@@ -66,22 +70,28 @@ object MetadataCleanerType extends Enumeration("MapOutputTracker", "SparkContext
def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString
}
+// TODO: This mutates a Conf to set properties right now, which is kind of ugly when used in the
+// initialization of StreamingContext. It's okay for users trying to configure stuff themselves.
object MetadataCleaner {
+ def getDelaySeconds(conf: SparkConf) = {
+ conf.get("spark.cleaner.ttl", "3500").toInt
+ }
- // 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 getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = {
- System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt
+ def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int =
+ {
+ conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString)
+ .toInt
}
- def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) {
- System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString)
+ def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType,
+ delay: Int)
+ {
+ conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString)
}
- def setDelaySeconds(delay: Int, resetAll: Boolean = true) {
+ def setDelaySeconds(conf: SparkConf, delay: Int, resetAll: Boolean = true) {
// override for all ?
- System.setProperty("spark.cleaner.ttl", delay.toString)
+ conf.set("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/SerializableHyperLogLog.scala b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala
new file mode 100644
index 0000000000..8b4e7c104c
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/SerializableHyperLogLog.scala
@@ -0,0 +1,50 @@
+/*
+ * 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 java.io.{Externalizable, ObjectOutput, ObjectInput}
+import com.clearspring.analytics.stream.cardinality.{ICardinality, HyperLogLog}
+
+/**
+ * A wrapper around [[com.clearspring.analytics.stream.cardinality.HyperLogLog]] that is serializable.
+ */
+private[spark]
+class SerializableHyperLogLog(var value: ICardinality) extends Externalizable {
+
+ def this() = this(null) // For deserialization
+
+ def merge(other: SerializableHyperLogLog) = new SerializableHyperLogLog(value.merge(other.value))
+
+ def add[T](elem: T) = {
+ this.value.offer(elem)
+ this
+ }
+
+ def readExternal(in: ObjectInput) {
+ val byteLength = in.readInt()
+ val bytes = new Array[Byte](byteLength)
+ in.readFully(bytes)
+ value = HyperLogLog.Builder.build(bytes)
+ }
+
+ def writeExternal(out: ObjectOutput) {
+ val bytes = value.getBytes()
+ out.writeInt(bytes.length)
+ out.write(bytes)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
index a25b37a2a9..bddb3bb735 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -30,10 +30,10 @@ import java.lang.management.ManagementFactory
import scala.collection.mutable.ArrayBuffer
import it.unimi.dsi.fastutil.ints.IntOpenHashSet
-import org.apache.spark.Logging
+import org.apache.spark.{SparkEnv, SparkConf, SparkContext, Logging}
/**
- * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in
+ * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in
* memory-aware caches.
*
* Based on the following JavaWorld article:
@@ -89,9 +89,11 @@ private[spark] object SizeEstimator extends Logging {
classInfos.put(classOf[Object], new ClassInfo(objectSize, Nil))
}
- private def getIsCompressedOops : Boolean = {
+ private def getIsCompressedOops: Boolean = {
+ // This is only used by tests to override the detection of compressed oops. The test
+ // actually uses a system property instead of a SparkConf, so we'll stick with that.
if (System.getProperty("spark.test.useCompressedOops") != null) {
- return System.getProperty("spark.test.useCompressedOops").toBoolean
+ return System.getProperty("spark.test.useCompressedOops").toBoolean
}
try {
@@ -103,7 +105,7 @@ private[spark] object SizeEstimator extends Logging {
val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption",
Class.forName("java.lang.String"))
- val bean = ManagementFactory.newPlatformMXBeanProxy(server,
+ val bean = ManagementFactory.newPlatformMXBeanProxy(server,
hotSpotMBeanName, hotSpotMBeanClass)
// TODO: We could use reflection on the VMOption returned ?
return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true")
@@ -251,7 +253,7 @@ private[spark] object SizeEstimator extends Logging {
if (info != null) {
return info
}
-
+
val parent = getClassInfo(cls.getSuperclass)
var shellSize = parent.shellSize
var pointerFields = parent.pointerFields
diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
index 277de2f8a6..181ae2fd45 100644
--- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala
@@ -85,7 +85,7 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with Logging {
}
override def filter(p: ((A, B)) => Boolean): Map[A, B] = {
- JavaConversions.asScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p)
+ JavaConversions.mapAsScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p)
}
override def empty: Map[A, B] = new TimeStampedHashMap[A, B]()
@@ -104,19 +104,28 @@ class TimeStampedHashMap[A, B] extends Map[A, B]() with Logging {
def toMap: immutable.Map[A, B] = iterator.toMap
/**
- * Removes old key-value pairs that have timestamp earlier than `threshTime`
+ * Removes old key-value pairs that have timestamp earlier than `threshTime`,
+ * calling the supplied function on each such entry before removing.
*/
- def clearOldValues(threshTime: Long) {
+ def clearOldValues(threshTime: Long, f: (A, B) => Unit) {
val iterator = internalMap.entrySet().iterator()
- while(iterator.hasNext) {
+ while (iterator.hasNext) {
val entry = iterator.next()
if (entry.getValue._2 < threshTime) {
+ f(entry.getKey, entry.getValue._1)
logDebug("Removing key " + entry.getKey)
iterator.remove()
}
}
}
+ /**
+ * Removes old key-value pairs that have timestamp earlier than `threshTime`
+ */
+ def clearOldValues(threshTime: Long) {
+ clearOldValues(threshTime, (_, _) => ())
+ }
+
private def currentTime: Long = System.currentTimeMillis()
}
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 a79e64e810..5f1253100b 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -22,10 +22,11 @@ import java.net.{InetAddress, URL, URI, NetworkInterface, Inet4Address}
import java.util.{Locale, Random, UUID}
import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor}
+import scala.collection.JavaConversions._
import scala.collection.Map
import scala.collection.mutable.ArrayBuffer
-import scala.collection.JavaConversions._
import scala.io.Source
+import scala.reflect.ClassTag
import com.google.common.io.Files
import com.google.common.util.concurrent.ThreadFactoryBuilder
@@ -35,14 +36,13 @@ import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance}
import org.apache.spark.deploy.SparkHadoopUtil
import java.nio.ByteBuffer
-import org.apache.spark.{SparkException, Logging}
+import org.apache.spark.{SparkConf, SparkContext, SparkException, Logging}
/**
* Various utility methods used by Spark.
*/
private[spark] object Utils extends Logging {
-
/** Serialize an object using Java serialization */
def serialize[T](o: T): Array[Byte] = {
val bos = new ByteArrayOutputStream()
@@ -238,9 +238,9 @@ private[spark] object Utils extends Logging {
* Throws SparkException if the target file already exists and has different contents than
* the requested file.
*/
- def fetchFile(url: String, targetDir: File) {
+ def fetchFile(url: String, targetDir: File, conf: SparkConf) {
val filename = url.split("/").last
- val tempDir = getLocalDir
+ val tempDir = getLocalDir(conf)
val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir))
val targetFile = new File(targetDir, filename)
val uri = new URI(url)
@@ -310,8 +310,8 @@ private[spark] object Utils extends Logging {
* return a single directory, even though the spark.local.dir property might be a list of
* multiple paths.
*/
- def getLocalDir: String = {
- System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0)
+ def getLocalDir(conf: SparkConf): String = {
+ conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0)
}
/**
@@ -319,7 +319,7 @@ private[spark] object Utils extends Logging {
* result in a new collection. Unlike scala.util.Random.shuffle, this method
* uses a local random number generator, avoiding inter-thread contention.
*/
- def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = {
+ def randomize[T: ClassTag](seq: TraversableOnce[T]): Seq[T] = {
randomizeInPlace(seq.toArray)
}
@@ -396,13 +396,12 @@ private[spark] object Utils extends Logging {
InetAddress.getByName(address).getHostName
}
- def localHostPort(): String = {
- val retval = System.getProperty("spark.hostPort", null)
+ def localHostPort(conf: SparkConf): String = {
+ val retval = conf.get("spark.hostPort", null)
if (retval == null) {
logErrorWithStack("spark.hostPort not set but invoking localHostPort")
return localHostName()
}
-
retval
}
@@ -414,9 +413,12 @@ private[spark] object Utils extends Logging {
assert(hostPort.indexOf(':') != -1, message)
}
- // Used by DEBUG code : remove when all testing done
def logErrorWithStack(msg: String) {
- try { throw new Exception } catch { case ex: Exception => { logError(msg, ex) } }
+ try {
+ throw new Exception
+ } catch {
+ case ex: Exception => logError(msg, ex)
+ }
}
// Typically, this will be of order of number of nodes in cluster
@@ -836,7 +838,7 @@ private[spark] object Utils extends Logging {
}
}
- /**
+ /**
* Timing method based on iterations that permit JVM JIT optimization.
* @param numIters number of iterations
* @param f function to be executed
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
index 1e9faaa5a0..a7a6635dec 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala
@@ -17,6 +17,7 @@
package org.apache.spark.util.collection
+import scala.reflect.ClassTag
/**
* A fast hash map implementation for nullable keys. This hash map supports insertions and updates,
@@ -26,7 +27,7 @@ package org.apache.spark.util.collection
* Under the hood, it uses our OpenHashSet implementation.
*/
private[spark]
-class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V: ClassManifest](
+class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag](
val keySet: OpenHashSet[K], var _values: Array[V])
extends Iterable[(K, V)]
with Serializable {
@@ -169,4 +170,3 @@ class OpenHashMap[K >: Null : ClassManifest, @specialized(Long, Int, Double) V:
_values(newPos) = _oldValues(oldPos)
}
}
-
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
index 36e2a05b9c..c908512b0f 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
@@ -17,6 +17,7 @@
package org.apache.spark.util.collection
+import scala.reflect._
/**
* A simple, fast hash set optimized for non-null insertion-only use case, where keys are never
@@ -36,7 +37,7 @@ package org.apache.spark.util.collection
* to explore all spaces for each key (see http://en.wikipedia.org/wiki/Quadratic_probing).
*/
private[spark]
-class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
+class OpenHashSet[@specialized(Long, Int) T: ClassTag](
initialCapacity: Int,
loadFactor: Double)
extends Serializable {
@@ -62,14 +63,14 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
// throws:
// scala.tools.nsc.symtab.Types$TypeError: type mismatch;
// found : scala.reflect.AnyValManifest[Long]
- // required: scala.reflect.ClassManifest[Int]
+ // required: scala.reflect.ClassTag[Int]
// at scala.tools.nsc.typechecker.Contexts$Context.error(Contexts.scala:298)
// at scala.tools.nsc.typechecker.Infer$Inferencer.error(Infer.scala:207)
// ...
- val mt = classManifest[T]
- if (mt == ClassManifest.Long) {
+ val mt = classTag[T]
+ if (mt == ClassTag.Long) {
(new LongHasher).asInstanceOf[Hasher[T]]
- } else if (mt == ClassManifest.Int) {
+ } else if (mt == ClassTag.Int) {
(new IntHasher).asInstanceOf[Hasher[T]]
} else {
new Hasher[T]
@@ -79,6 +80,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
protected var _capacity = nextPowerOf2(initialCapacity)
protected var _mask = _capacity - 1
protected var _size = 0
+ protected var _growThreshold = (loadFactor * _capacity).toInt
protected var _bitset = new BitSet(_capacity)
@@ -117,7 +119,29 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
* @return The position where the key is placed, plus the highest order bit is set if the key
* exists previously.
*/
- def addWithoutResize(k: T): Int = putInto(_bitset, _data, k)
+ def addWithoutResize(k: T): Int = {
+ var pos = hashcode(hasher.hash(k)) & _mask
+ var i = 1
+ while (true) {
+ if (!_bitset.get(pos)) {
+ // This is a new key.
+ _data(pos) = k
+ _bitset.set(pos)
+ _size += 1
+ return pos | NONEXISTENCE_MASK
+ } else if (_data(pos) == k) {
+ // Found an existing key.
+ return pos
+ } else {
+ val delta = i
+ pos = (pos + delta) & _mask
+ i += 1
+ }
+ }
+ // Never reached here
+ assert(INVALID_POS != INVALID_POS)
+ INVALID_POS
+ }
/**
* Rehash the set if it is overloaded.
@@ -128,7 +152,7 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
* to a new position (in the new data array).
*/
def rehashIfNeeded(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) {
- if (_size > loadFactor * _capacity) {
+ if (_size > _growThreshold) {
rehash(k, allocateFunc, moveFunc)
}
}
@@ -181,37 +205,6 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
def nextPos(fromPos: Int): Int = _bitset.nextSetBit(fromPos)
/**
- * Put an entry into the set. Return the position where the key is placed. In addition, the
- * highest bit in the returned position is set if the key exists prior to this put.
- *
- * This function assumes the data array has at least one empty slot.
- */
- private def putInto(bitset: BitSet, data: Array[T], k: T): Int = {
- val mask = data.length - 1
- var pos = hashcode(hasher.hash(k)) & mask
- var i = 1
- while (true) {
- if (!bitset.get(pos)) {
- // This is a new key.
- data(pos) = k
- bitset.set(pos)
- _size += 1
- return pos | NONEXISTENCE_MASK
- } else if (data(pos) == k) {
- // Found an existing key.
- return pos
- } else {
- val delta = i
- pos = (pos + delta) & mask
- i += 1
- }
- }
- // Never reached here
- assert(INVALID_POS != INVALID_POS)
- INVALID_POS
- }
-
- /**
* Double the table's size and re-hash everything. We are not really using k, but it is declared
* so Scala compiler can specialize this method (which leads to calling the specialized version
* of putInto).
@@ -224,33 +217,49 @@ class OpenHashSet[@specialized(Long, Int) T: ClassManifest](
*/
private def rehash(k: T, allocateFunc: (Int) => Unit, moveFunc: (Int, Int) => Unit) {
val newCapacity = _capacity * 2
- require(newCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements")
-
allocateFunc(newCapacity)
- val newData = new Array[T](newCapacity)
val newBitset = new BitSet(newCapacity)
- var pos = 0
- _size = 0
- while (pos < _capacity) {
- if (_bitset.get(pos)) {
- val newPos = putInto(newBitset, newData, _data(pos))
- moveFunc(pos, newPos & POSITION_MASK)
+ val newData = new Array[T](newCapacity)
+ val newMask = newCapacity - 1
+
+ var oldPos = 0
+ while (oldPos < capacity) {
+ if (_bitset.get(oldPos)) {
+ val key = _data(oldPos)
+ var newPos = hashcode(hasher.hash(key)) & newMask
+ var i = 1
+ var keepGoing = true
+ // No need to check for equality here when we insert so this has one less if branch than
+ // the similar code path in addWithoutResize.
+ while (keepGoing) {
+ if (!newBitset.get(newPos)) {
+ // Inserting the key at newPos
+ newData(newPos) = key
+ newBitset.set(newPos)
+ moveFunc(oldPos, newPos)
+ keepGoing = false
+ } else {
+ val delta = i
+ newPos = (newPos + delta) & newMask
+ i += 1
+ }
+ }
}
- pos += 1
+ oldPos += 1
}
+
_bitset = newBitset
_data = newData
_capacity = newCapacity
- _mask = newCapacity - 1
+ _mask = newMask
+ _growThreshold = (loadFactor * newCapacity).toInt
}
/**
- * Re-hash a value to deal better with hash functions that don't differ
- * in the lower bits, similar to java.util.HashMap
+ * Re-hash a value to deal better with hash functions that don't differ in the lower bits.
+ * We use the Murmur Hash 3 finalization step that's also used in fastutil.
*/
- private def hashcode(h: Int): Int = {
- it.unimi.dsi.fastutil.HashCommon.murmurHash3(h)
- }
+ private def hashcode(h: Int): Int = it.unimi.dsi.fastutil.HashCommon.murmurHash3(h)
private def nextPowerOf2(n: Int): Int = {
val highBit = Integer.highestOneBit(n)
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
index d6a3cdb405..1dc9f744e1 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMap.scala
@@ -17,6 +17,7 @@
package org.apache.spark.util.collection
+import scala.reflect._
/**
* A fast hash map implementation for primitive, non-null keys. This hash map supports
@@ -26,8 +27,8 @@ package org.apache.spark.util.collection
* Under the hood, it uses our OpenHashSet implementation.
*/
private[spark]
-class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest,
- @specialized(Long, Int, Double) V: ClassManifest](
+class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassTag,
+ @specialized(Long, Int, Double) V: ClassTag](
val keySet: OpenHashSet[K], var _values: Array[V])
extends Iterable[(K, V)]
with Serializable {
@@ -37,20 +38,19 @@ class PrimitiveKeyOpenHashMap[@specialized(Long, Int) K: ClassManifest,
*/
def this(initialCapacity: Int) =
this(new OpenHashSet[K](initialCapacity), new Array[V](initialCapacity))
-
+
/**
* Allocate an OpenHashMap with a default initial capacity, providing a true
* no-argument constructor.
*/
def this() = this(64)
-
/**
* Allocate an OpenHashMap with a fixed initial capacity
*/
def this(keySet: OpenHashSet[K]) = this(keySet, new Array[V](keySet.capacity))
- require(classManifest[K] == classManifest[Long] || classManifest[K] == classManifest[Int])
+ require(classTag[K] == classTag[Long] || classTag[K] == classTag[Int])
private var _oldValues: Array[V] = null
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
index 20554f0aab..b84eb65c62 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
@@ -17,11 +17,13 @@
package org.apache.spark.util.collection
+import scala.reflect.ClassTag
+
/**
* An append-only, non-threadsafe, array-backed vector that is optimized for primitive types.
*/
private[spark]
-class PrimitiveVector[@specialized(Long, Int, Double) V: ClassManifest](initialSize: Int = 64) {
+class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: Int = 64) {
private var _numElements = 0
private var _array: Array[V] = _
diff --git a/core/src/test/resources/spark.conf b/core/src/test/resources/spark.conf
new file mode 100644
index 0000000000..aa4e751235
--- /dev/null
+++ b/core/src/test/resources/spark.conf
@@ -0,0 +1,8 @@
+# A simple spark.conf file used only in our unit tests
+
+spark.test.intTestProperty = 1
+
+spark.test {
+ stringTestProperty = "hi"
+ listTestProperty = ["a", "b"]
+}
diff --git a/core/src/test/resources/uncommons-maths-1.2.2.jar b/core/src/test/resources/uncommons-maths-1.2.2.jar
deleted file mode 100644
index e126001c1c..0000000000
--- a/core/src/test/resources/uncommons-maths-1.2.2.jar
+++ /dev/null
Binary files differ
diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index 4434f3b87c..c443c5266e 100644
--- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -27,6 +27,21 @@ import org.apache.spark.SparkContext._
class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
+
+ implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] {
+ def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = {
+ t1 ++= t2
+ t1
+ }
+ def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = {
+ t1 += t2
+ t1
+ }
+ def zero(t: mutable.Set[A]) : mutable.Set[A] = {
+ new mutable.HashSet[A]()
+ }
+ }
+
test ("basic accumulation"){
sc = new SparkContext("local", "test")
val acc : Accumulator[Int] = sc.accumulator(0)
@@ -51,7 +66,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte
}
test ("add value to collection accumulators") {
- import SetAccum._
val maxI = 1000
for (nThreads <- List(1, 10)) { //test single & multi-threaded
sc = new SparkContext("local[" + nThreads + "]", "test")
@@ -68,22 +82,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte
}
}
- implicit object SetAccum extends AccumulableParam[mutable.Set[Any], Any] {
- def addInPlace(t1: mutable.Set[Any], t2: mutable.Set[Any]) : mutable.Set[Any] = {
- t1 ++= t2
- t1
- }
- def addAccumulator(t1: mutable.Set[Any], t2: Any) : mutable.Set[Any] = {
- t1 += t2
- t1
- }
- def zero(t: mutable.Set[Any]) : mutable.Set[Any] = {
- new mutable.HashSet[Any]()
- }
- }
-
test ("value not readable in tasks") {
- import SetAccum._
val maxI = 1000
for (nThreads <- List(1, 10)) { //test single & multi-threaded
sc = new SparkContext("local[" + nThreads + "]", "test")
@@ -125,7 +124,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte
}
test ("localValue readable in tasks") {
- import SetAccum._
val maxI = 1000
for (nThreads <- List(1, 10)) { //test single & multi-threaded
sc = new SparkContext("local[" + nThreads + "]", "test")
diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
index f26c44d3e7..ec13b329b2 100644
--- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark
+import scala.reflect.ClassTag
import org.scalatest.FunSuite
import java.io.File
import org.apache.spark.rdd._
@@ -25,8 +26,6 @@ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId}
import org.apache.spark.util.Utils
class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
- initLogging()
-
var checkpointDir: File = _
val partitioner = new HashPartitioner(2)
@@ -56,17 +55,15 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
}
test("RDDs with one-to-one dependencies") {
- testCheckpointing(_.map(x => x.toString))
- testCheckpointing(_.flatMap(x => 1 to x))
- testCheckpointing(_.filter(_ % 2 == 0))
- testCheckpointing(_.sample(false, 0.5, 0))
- testCheckpointing(_.glom())
- testCheckpointing(_.mapPartitions(_.map(_.toString)))
- 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")))
+ testRDD(_.map(x => x.toString))
+ testRDD(_.flatMap(x => 1 to x))
+ testRDD(_.filter(_ % 2 == 0))
+ testRDD(_.sample(false, 0.5, 0))
+ testRDD(_.glom())
+ testRDD(_.mapPartitions(_.map(_.toString)))
+ testRDD(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
+ testRDD(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
+ testRDD(_.pipe(Seq("cat")))
}
test("ParallelCollection") {
@@ -98,7 +95,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
}
test("ShuffledRDD") {
- testCheckpointing(rdd => {
+ testRDD(rdd => {
// Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD
new ShuffledRDD[Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner)
})
@@ -106,25 +103,17 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
test("UnionRDD") {
def otherRDD = sc.makeRDD(1 to 10, 1)
-
- // Test whether the size of UnionRDDPartitions reduce in size after parent RDD is checkpointed.
- // Current implementation of UnionRDD has transient reference to parent RDDs,
- // so only the partitions will reduce in serialized size, not the RDD.
- testCheckpointing(_.union(otherRDD), false, true)
- testParentCheckpointing(_.union(otherRDD), false, true)
+ testRDD(_.union(otherRDD))
+ testRDDPartitions(_.union(otherRDD))
}
test("CartesianRDD") {
def otherRDD = sc.makeRDD(1 to 10, 1)
- testCheckpointing(new CartesianRDD(sc, _, otherRDD))
-
- // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
- // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
- // so only the RDD will reduce in serialized size, not the partitions.
- testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false)
+ testRDD(new CartesianRDD(sc, _, otherRDD))
+ testRDDPartitions(new CartesianRDD(sc, _, otherRDD))
// Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after
- // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
+ // the parent RDD has been checkpointed and parent partitions have been changed.
// Note that this test is very specific to the current implementation of CartesianRDD.
val ones = sc.makeRDD(1 to 100, 10).map(x => x)
ones.checkpoint() // checkpoint that MappedRDD
@@ -135,23 +124,20 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
val splitAfterCheckpoint =
serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition])
assert(
- (splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) &&
- (splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2),
- "CartesianRDD.parents not updated after parent RDD checkpointed"
+ (splitAfterCheckpoint.s1.getClass != splitBeforeCheckpoint.s1.getClass) &&
+ (splitAfterCheckpoint.s2.getClass != splitBeforeCheckpoint.s2.getClass),
+ "CartesianRDD.s1 and CartesianRDD.s2 not updated after parent RDD is checkpointed"
)
}
test("CoalescedRDD") {
- testCheckpointing(_.coalesce(2))
-
- // Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
- // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
- // so only the RDD will reduce in serialized size, not the partitions.
- testParentCheckpointing(_.coalesce(2), true, false)
+ testRDD(_.coalesce(2))
+ testRDDPartitions(_.coalesce(2))
- // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents) after
- // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
- // Note that this test is very specific to the current implementation of CoalescedRDDPartitions
+ // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents)
+ // after the parent RDD has been checkpointed and parent partitions have been changed.
+ // Note that this test is very specific to the current implementation of
+ // CoalescedRDDPartitions.
val ones = sc.makeRDD(1 to 100, 10).map(x => x)
ones.checkpoint() // checkpoint that MappedRDD
val coalesced = new CoalescedRDD(ones, 2)
@@ -161,33 +147,78 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
val splitAfterCheckpoint =
serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition])
assert(
- splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head,
- "CoalescedRDDPartition.parents not updated after parent RDD checkpointed"
+ splitAfterCheckpoint.parents.head.getClass != splitBeforeCheckpoint.parents.head.getClass,
+ "CoalescedRDDPartition.parents not updated after parent RDD is checkpointed"
)
}
test("CoGroupedRDD") {
- val longLineageRDD1 = generateLongLineageRDDForCoGroupedRDD()
- testCheckpointing(rdd => {
+ val longLineageRDD1 = generateFatPairRDD()
+ testRDD(rdd => {
CheckpointSuite.cogroup(longLineageRDD1, rdd.map(x => (x % 2, 1)), partitioner)
- }, false, true)
+ })
- val longLineageRDD2 = generateLongLineageRDDForCoGroupedRDD()
- testParentCheckpointing(rdd => {
+ val longLineageRDD2 = generateFatPairRDD()
+ testRDDPartitions(rdd => {
CheckpointSuite.cogroup(
longLineageRDD2, sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)), partitioner)
- }, false, true)
+ })
}
test("ZippedRDD") {
- testCheckpointing(
- rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
-
- // Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed
- // Current implementation of ZippedRDDPartitions has transient references to parent RDDs,
- // so only the RDD will reduce in serialized size, not the partitions.
- testParentCheckpointing(
- rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
+ testRDD(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)))
+ testRDDPartitions(rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)))
+
+ // Test that the ZippedPartition updates parent partitions
+ // after the parent RDD has been checkpointed and parent partitions have been changed.
+ // Note that this test is very specific to the current implementation of ZippedRDD.
+ val rdd = generateFatRDD()
+ val zippedRDD = new ZippedRDD(sc, rdd, rdd.map(x => x))
+ zippedRDD.rdd1.checkpoint()
+ zippedRDD.rdd2.checkpoint()
+ val partitionBeforeCheckpoint =
+ serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]])
+ zippedRDD.count()
+ val partitionAfterCheckpoint =
+ serializeDeserialize(zippedRDD.partitions.head.asInstanceOf[ZippedPartition[_, _]])
+ assert(
+ partitionAfterCheckpoint.partition1.getClass != partitionBeforeCheckpoint.partition1.getClass &&
+ partitionAfterCheckpoint.partition2.getClass != partitionBeforeCheckpoint.partition2.getClass,
+ "ZippedRDD.partition1 and ZippedRDD.partition2 not updated after parent RDD is checkpointed"
+ )
+ }
+
+ test("PartitionerAwareUnionRDD") {
+ testRDD(rdd => {
+ new PartitionerAwareUnionRDD[(Int, Int)](sc, Array(
+ generateFatPairRDD(),
+ rdd.map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
+ ))
+ })
+
+ testRDDPartitions(rdd => {
+ new PartitionerAwareUnionRDD[(Int, Int)](sc, Array(
+ generateFatPairRDD(),
+ rdd.map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
+ ))
+ })
+
+ // Test that the PartitionerAwareUnionRDD updates parent partitions
+ // (PartitionerAwareUnionRDD.parents) after the parent RDD has been checkpointed and parent
+ // partitions have been changed. Note that this test is very specific to the current
+ // implementation of PartitionerAwareUnionRDD.
+ val pairRDD = generateFatPairRDD()
+ pairRDD.checkpoint()
+ val unionRDD = new PartitionerAwareUnionRDD(sc, Array(pairRDD))
+ val partitionBeforeCheckpoint = serializeDeserialize(
+ unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition])
+ pairRDD.count()
+ val partitionAfterCheckpoint = serializeDeserialize(
+ unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition])
+ assert(
+ partitionBeforeCheckpoint.parents.head.getClass != partitionAfterCheckpoint.parents.head.getClass,
+ "PartitionerAwareUnionRDDPartition.parents not updated after parent RDD is checkpointed"
+ )
}
test("CheckpointRDD with zero partitions") {
@@ -201,29 +232,32 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
}
/**
- * Test checkpointing of the final RDD generated by the given operation. By default,
- * this method tests whether the size of serialized RDD has reduced after checkpointing or not.
- * It can also test whether the size of serialized RDD partitions has reduced after checkpointing or
- * not, but this is not done by default as usually the partitions do not refer to any RDD and
- * therefore never store the lineage.
+ * Test checkpointing of the RDD generated by the given operation. It tests whether the
+ * serialized size of the RDD is reduce after checkpointing or not. This function should be called
+ * on all RDDs that have a parent RDD (i.e., do not call on ParallelCollection, BlockRDD, etc.).
*/
- def testCheckpointing[U: ClassManifest](
- op: (RDD[Int]) => RDD[U],
- testRDDSize: Boolean = true,
- testRDDPartitionSize: Boolean = false
- ) {
+ def testRDD[U: ClassTag](op: (RDD[Int]) => RDD[U]) {
// Generate the final RDD using given RDD operation
- val baseRDD = generateLongLineageRDD()
+ val baseRDD = generateFatRDD()
val operatedRDD = op(baseRDD)
val parentRDD = operatedRDD.dependencies.headOption.orNull
val rddType = operatedRDD.getClass.getSimpleName
val numPartitions = operatedRDD.partitions.length
+ // Force initialization of all the data structures in RDDs
+ // Without this, serializing the RDD will give a wrong estimate of the size of the RDD
+ initializeRdd(operatedRDD)
+
+ val partitionsBeforeCheckpoint = operatedRDD.partitions
+
// Find serialized sizes before and after the checkpoint
- val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+ logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
+ val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
operatedRDD.checkpoint()
val result = operatedRDD.collect()
- val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+ operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables
+ val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+ logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
// Test whether the checkpoint file has been created
assert(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get).collect() === result)
@@ -231,6 +265,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
// Test whether dependencies have been changed from its earlier parent RDD
assert(operatedRDD.dependencies.head.rdd != parentRDD)
+ // Test whether the partitions have been changed from its earlier partitions
+ assert(operatedRDD.partitions.toList != partitionsBeforeCheckpoint.toList)
+
// Test whether the partitions have been changed to the new Hadoop partitions
assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList)
@@ -240,122 +277,72 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
// Test whether the data in the checkpointed RDD is same as original
assert(operatedRDD.collect() === result)
- // Test whether serialized size of the RDD has reduced. If the RDD
- // does not have any dependency to another RDD (e.g., ParallelCollection,
- // ShuffleRDD with ShuffleDependency), it may not reduce in size after checkpointing.
- if (testRDDSize) {
- logInfo("Size of " + rddType +
- "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]")
- assert(
- rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
- "Size of " + rddType + " did not reduce after checkpointing " +
- "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
- )
- }
+ // Test whether serialized size of the RDD has reduced.
+ logInfo("Size of " + rddType +
+ " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]")
+ assert(
+ rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
+ "Size of " + rddType + " did not reduce after checkpointing " +
+ " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
+ )
- // Test whether serialized size of the partitions has reduced. If the partitions
- // do not have any non-transient reference to another RDD or another RDD's partitions, it
- // does not refer to a lineage and therefore may not reduce in size after checkpointing.
- // However, if the original partitions before checkpointing do refer to a parent RDD, the partitions
- // must be forgotten after checkpointing (to remove all reference to parent RDDs) and
- // replaced with the HadooPartitions of the checkpointed RDD.
- if (testRDDPartitionSize) {
- logInfo("Size of " + rddType + " partitions "
- + "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]")
- assert(
- splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
- "Size of " + rddType + " partitions did not reduce after checkpointing " +
- "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
- )
- }
}
/**
* Test whether checkpointing of the parent of the generated RDD also
* truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent
* RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed,
- * this RDD will remember the partitions and therefore potentially the whole lineage.
+ * the generated RDD will remember the partitions and therefore potentially the whole lineage.
+ * This function should be called only those RDD whose partitions refer to parent RDD's
+ * partitions (i.e., do not call it on simple RDD like MappedRDD).
+ *
*/
- def testParentCheckpointing[U: ClassManifest](
- op: (RDD[Int]) => RDD[U],
- testRDDSize: Boolean,
- testRDDPartitionSize: Boolean
- ) {
+ def testRDDPartitions[U: ClassTag](op: (RDD[Int]) => RDD[U]) {
// Generate the final RDD using given RDD operation
- val baseRDD = generateLongLineageRDD()
+ val baseRDD = generateFatRDD()
val operatedRDD = op(baseRDD)
- val parentRDD = operatedRDD.dependencies.head.rdd
+ val parentRDDs = operatedRDD.dependencies.map(_.rdd)
val rddType = operatedRDD.getClass.getSimpleName
- val parentRDDType = parentRDD.getClass.getSimpleName
- // Get the partitions and dependencies of the parent in case they're lazily computed
- parentRDD.dependencies
- parentRDD.partitions
+ // Force initialization of all the data structures in RDDs
+ // Without this, serializing the RDD will give a wrong estimate of the size of the RDD
+ initializeRdd(operatedRDD)
// Find serialized sizes before and after the checkpoint
- val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
- parentRDD.checkpoint() // checkpoint the parent RDD, not the generated one
- val result = operatedRDD.collect()
- val (rddSizeAfterCheckpoint, splitSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+ logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
+ val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
+ parentRDDs.foreach(_.checkpoint()) // checkpoint the parent RDD, not the generated one
+ val result = operatedRDD.collect() // force checkpointing
+ operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables
+ val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD)
+ logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString)
// Test whether the data in the checkpointed RDD is same as original
assert(operatedRDD.collect() === result)
- // Test whether serialized size of the RDD has reduced because of its parent being
- // checkpointed. If this RDD or its parent RDD do not have any dependency
- // to another RDD (e.g., ParallelCollection, ShuffleRDD with ShuffleDependency), it may
- // not reduce in size after checkpointing.
- if (testRDDSize) {
- assert(
- rddSizeAfterCheckpoint < rddSizeBeforeCheckpoint,
- "Size of " + rddType + " did not reduce after checkpointing parent " + parentRDDType +
- "[" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]"
- )
- }
-
- // Test whether serialized size of the partitions has reduced because of its parent being
- // checkpointed. If the partitions do not have any non-transient reference to another RDD
- // or another RDD's partitions, it does not refer to a lineage and therefore may not reduce
- // in size after checkpointing. However, if the partitions do refer to the *partitions* of a parent
- // RDD, then these partitions must update reference to the parent RDD partitions as the parent RDD's
- // partitions must have changed after checkpointing.
- if (testRDDPartitionSize) {
- assert(
- splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
- "Size of " + rddType + " partitions did not reduce after checkpointing parent " + parentRDDType +
- "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
- )
- }
-
+ // Test whether serialized size of the partitions has reduced
+ logInfo("Size of partitions of " + rddType +
+ " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]")
+ assert(
+ partitionSizeAfterCheckpoint < partitionSizeBeforeCheckpoint,
+ "Size of " + rddType + " partitions did not reduce after checkpointing parent RDDs" +
+ " [" + partitionSizeBeforeCheckpoint + " --> " + partitionSizeAfterCheckpoint + "]"
+ )
}
/**
- * Generate an RDD with a long lineage of one-to-one dependencies.
+ * Generate an RDD such that both the RDD and its partitions have large size.
*/
- def generateLongLineageRDD(): RDD[Int] = {
- var rdd = sc.makeRDD(1 to 100, 4)
- for (i <- 1 to 50) {
- rdd = rdd.map(x => x + 1)
- }
- rdd
+ def generateFatRDD(): RDD[Int] = {
+ new FatRDD(sc.makeRDD(1 to 100, 4)).map(x => x)
}
/**
- * Generate an RDD with a long lineage specifically for CoGroupedRDD.
- * A CoGroupedRDD can have a long lineage only one of its parents have a long lineage
- * and narrow dependency with this RDD. This method generate such an RDD by a sequence
- * of cogroups and mapValues which creates a long lineage of narrow dependencies.
+ * Generate an pair RDD (with partitioner) such that both the RDD and its partitions
+ * have large size.
*/
- def generateLongLineageRDDForCoGroupedRDD() = {
- val add = (x: (Seq[Int], Seq[Int])) => (x._1 ++ x._2).reduce(_ + _)
-
- def ones: RDD[(Int, Int)] = sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)).reduceByKey(partitioner, _ + _)
-
- var cogrouped: RDD[(Int, (Seq[Int], Seq[Int]))] = ones.cogroup(ones)
- for(i <- 1 to 10) {
- cogrouped = cogrouped.mapValues(add).cogroup(ones)
- }
- cogrouped.mapValues(add)
+ def generateFatPairRDD() = {
+ new FatPairRDD(sc.makeRDD(1 to 100, 4), partitioner).mapValues(x => x)
}
/**
@@ -363,8 +350,26 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
* upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint.
*/
def getSerializedSizes(rdd: RDD[_]): (Int, Int) = {
- (Utils.serialize(rdd).length - Utils.serialize(rdd.checkpointData).length,
- Utils.serialize(rdd.partitions).length)
+ val rddSize = Utils.serialize(rdd).size
+ val rddCpDataSize = Utils.serialize(rdd.checkpointData).size
+ val rddPartitionSize = Utils.serialize(rdd.partitions).size
+ val rddDependenciesSize = Utils.serialize(rdd.dependencies).size
+
+ // Print detailed size, helps in debugging
+ logInfo("Serialized sizes of " + rdd +
+ ": RDD = " + rddSize +
+ ", RDD checkpoint data = " + rddCpDataSize +
+ ", RDD partitions = " + rddPartitionSize +
+ ", RDD dependencies = " + rddDependenciesSize
+ )
+ // this makes sure that serializing the RDD's checkpoint data does not
+ // serialize the whole RDD as well
+ assert(
+ rddSize > rddCpDataSize,
+ "RDD's checkpoint data (" + rddCpDataSize + ") is equal or larger than the " +
+ "whole RDD with checkpoint data (" + rddSize + ")"
+ )
+ (rddSize - rddCpDataSize, rddPartitionSize)
}
/**
@@ -376,8 +381,49 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
val bytes = Utils.serialize(obj)
Utils.deserialize[T](bytes)
}
+
+ /**
+ * Recursively force the initialization of the all members of an RDD and it parents.
+ */
+ def initializeRdd(rdd: RDD[_]) {
+ rdd.partitions // forces the
+ rdd.dependencies.map(_.rdd).foreach(initializeRdd(_))
+ }
}
+/** RDD partition that has large serialized size. */
+class FatPartition(val partition: Partition) extends Partition {
+ val bigData = new Array[Byte](10000)
+ def index: Int = partition.index
+}
+
+/** RDD that has large serialized size. */
+class FatRDD(parent: RDD[Int]) extends RDD[Int](parent) {
+ val bigData = new Array[Byte](100000)
+
+ protected def getPartitions: Array[Partition] = {
+ parent.partitions.map(p => new FatPartition(p))
+ }
+
+ def compute(split: Partition, context: TaskContext): Iterator[Int] = {
+ parent.compute(split.asInstanceOf[FatPartition].partition, context)
+ }
+}
+
+/** Pair RDD that has large serialized size. */
+class FatPairRDD(parent: RDD[Int], _partitioner: Partitioner) extends RDD[(Int, Int)](parent) {
+ val bigData = new Array[Byte](100000)
+
+ protected def getPartitions: Array[Partition] = {
+ parent.partitions.map(p => new FatPartition(p))
+ }
+
+ @transient override val partitioner = Some(_partitioner)
+
+ def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = {
+ parent.compute(split.asInstanceOf[FatPartition].partition, context).map(x => (x, x))
+ }
+}
object CheckpointSuite {
// This is a custom cogroup function that does not use mapValues like
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index 480bac84f3..d9cb7fead5 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -122,7 +122,7 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
sc.parallelize(1 to 10, 10).foreach(x => println(x / 0))
}
assert(thrown.getClass === classOf[SparkException])
- assert(thrown.getMessage.contains("more than 4 times"))
+ assert(thrown.getMessage.contains("failed 4 times"))
}
test("caching") {
@@ -303,12 +303,13 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
Thread.sleep(200)
}
} catch {
- case _ => { Thread.sleep(10) }
+ case _: Throwable => { Thread.sleep(10) }
// Do nothing. We might see exceptions because block manager
// is racing this thread to remove entries from the driver.
}
}
}
+
}
object DistributedSuite {
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index 01a72d8401..fb89537258 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -30,13 +30,15 @@ import org.apache.spark.util.Utils
class DriverSuite extends FunSuite with Timeouts {
test("driver should exit after finishing") {
- assert(System.getenv("SPARK_HOME") != null)
+ val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get
// Regression test for SPARK-530: "Spark driver process doesn't exit after finishing"
val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]"))
forAll(masters) { (master: String) =>
- failAfter(30 seconds) {
- Utils.execute(Seq("./spark-class", "org.apache.spark.DriverWithoutCleanup", master),
- new File(System.getenv("SPARK_HOME")))
+ failAfter(60 seconds) {
+ Utils.executeAndGetOutput(
+ Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master),
+ new File(sparkHome),
+ Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
}
}
}
diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala
index af448fcb37..befdc1589f 100644
--- a/core/src/test/scala/org/apache/spark/FailureSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala
@@ -42,7 +42,7 @@ class FailureSuite extends FunSuite with LocalSparkContext {
// Run a 3-task map job in which task 1 deterministically fails once, and check
// whether the job completes successfully and we ran 4 tasks in total.
test("failure in a single-stage job") {
- sc = new SparkContext("local[1,1]", "test")
+ sc = new SparkContext("local[1,2]", "test")
val results = sc.makeRDD(1 to 3, 3).map { x =>
FailureSuiteState.synchronized {
FailureSuiteState.tasksRun += 1
@@ -62,7 +62,7 @@ class FailureSuite extends FunSuite with LocalSparkContext {
// Run a map-reduce job in which a reduce task deterministically fails once.
test("failure in a two-stage job") {
- sc = new SparkContext("local[1,1]", "test")
+ sc = new SparkContext("local[1,2]", "test")
val results = sc.makeRDD(1 to 3).map(x => (x, x)).groupByKey(3).map {
case (k, v) =>
FailureSuiteState.synchronized {
diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
index c210dd5c3b..a2eb9a4e84 100644
--- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala
@@ -17,33 +17,49 @@
package org.apache.spark
+import java.io._
+import java.util.jar.{JarEntry, JarOutputStream}
+
+import SparkContext._
import com.google.common.io.Files
import org.scalatest.FunSuite
-import java.io.{File, PrintWriter, FileReader, BufferedReader}
-import SparkContext._
class FileServerSuite extends FunSuite with LocalSparkContext {
@transient var tmpFile: File = _
- @transient var testJarFile: File = _
-
- override def beforeEach() {
- super.beforeEach()
- // Create a sample text file
- val tmpdir = new File(Files.createTempDir(), "test")
- tmpdir.mkdir()
- tmpFile = new File(tmpdir, "FileServerSuite.txt")
- val pw = new PrintWriter(tmpFile)
+ @transient var tmpJarUrl: String = _
+
+ override def beforeAll() {
+ super.beforeAll()
+ val tmpDir = new File(Files.createTempDir(), "test")
+ tmpDir.mkdir()
+
+ val textFile = new File(tmpDir, "FileServerSuite.txt")
+ val pw = new PrintWriter(textFile)
pw.println("100")
pw.close()
- }
+
+ val jarFile = new File(tmpDir, "test.jar")
+ val jarStream = new FileOutputStream(jarFile)
+ val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest())
- override def afterEach() {
- super.afterEach()
- // Clean up downloaded file
- if (tmpFile.exists) {
- tmpFile.delete()
+ val jarEntry = new JarEntry(textFile.getName)
+ jar.putNextEntry(jarEntry)
+
+ val in = new FileInputStream(textFile)
+ val buffer = new Array[Byte](10240)
+ var nRead = 0
+ while (nRead <= 0) {
+ nRead = in.read(buffer, 0, buffer.length)
+ jar.write(buffer, 0, nRead)
}
+
+ in.close()
+ jar.close()
+ jarStream.close()
+
+ tmpFile = textFile
+ tmpJarUrl = jarFile.toURI.toURL.toString
}
test("Distributing files locally") {
@@ -77,18 +93,13 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
test ("Dynamically adding JARS locally") {
sc = new SparkContext("local[4]", "test")
- val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile()
- sc.addJar(sampleJarFile)
- val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0))
- val result = sc.parallelize(testData).reduceByKey { (x,y) =>
- val fac = Thread.currentThread.getContextClassLoader()
- .loadClass("org.uncommons.maths.Maths")
- .getDeclaredMethod("factorial", classOf[Int])
- val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
- val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
- a + b
- }.collect()
- assert(result.toSet === Set((1,2), (2,7), (3,121)))
+ sc.addJar(tmpJarUrl)
+ val testData = Array((1, 1))
+ sc.parallelize(testData).foreach { x =>
+ if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) {
+ throw new SparkException("jar not added")
+ }
+ }
}
test("Distributing files on a standalone cluster") {
@@ -107,33 +118,24 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
test ("Dynamically adding JARS on a standalone cluster") {
sc = new SparkContext("local-cluster[1,1,512]", "test")
- val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile()
- sc.addJar(sampleJarFile)
- val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0))
- val result = sc.parallelize(testData).reduceByKey { (x,y) =>
- val fac = Thread.currentThread.getContextClassLoader()
- .loadClass("org.uncommons.maths.Maths")
- .getDeclaredMethod("factorial", classOf[Int])
- val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
- val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
- a + b
- }.collect()
- assert(result.toSet === Set((1,2), (2,7), (3,121)))
+ sc.addJar(tmpJarUrl)
+ val testData = Array((1,1))
+ sc.parallelize(testData).foreach { x =>
+ if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) {
+ throw new SparkException("jar not added")
+ }
+ }
}
test ("Dynamically adding JARS on a standalone cluster using local: URL") {
sc = new SparkContext("local-cluster[1,1,512]", "test")
- val sampleJarFile = getClass.getClassLoader.getResource("uncommons-maths-1.2.2.jar").getFile()
- sc.addJar(sampleJarFile.replace("file", "local"))
- val testData = Array((1,1), (1,1), (2,1), (3,5), (2,3), (3,0))
- val result = sc.parallelize(testData).reduceByKey { (x,y) =>
- val fac = Thread.currentThread.getContextClassLoader()
- .loadClass("org.uncommons.maths.Maths")
- .getDeclaredMethod("factorial", classOf[Int])
- val a = fac.invoke(null, x.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
- val b = fac.invoke(null, y.asInstanceOf[java.lang.Integer]).asInstanceOf[Long].toInt
- a + b
- }.collect()
- assert(result.toSet === Set((1,2), (2,7), (3,121)))
+ sc.addJar(tmpJarUrl.replace("file", "local"))
+ val testData = Array((1,1))
+ sc.parallelize(testData).foreach { x =>
+ if (Thread.currentThread.getContextClassLoader.getResource("FileServerSuite.txt") == null) {
+ throw new SparkException("jar not added")
+ }
+ }
}
+
}
diff --git a/core/src/test/scala/org/apache/spark/JavaAPISuite.java b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
index 352036f182..23ec6c3b31 100644
--- a/core/src/test/scala/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/scala/org/apache/spark/JavaAPISuite.java
@@ -365,6 +365,20 @@ public class JavaAPISuite implements Serializable {
}
@Test
+ public void javaDoubleRDDHistoGram() {
+ JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0));
+ // Test using generated buckets
+ Tuple2<double[], long[]> results = rdd.histogram(2);
+ double[] expected_buckets = {1.0, 2.5, 4.0};
+ long[] expected_counts = {2, 2};
+ Assert.assertArrayEquals(expected_buckets, results._1, 0.1);
+ Assert.assertArrayEquals(expected_counts, results._2);
+ // Test with provided buckets
+ long[] histogram = rdd.histogram(expected_buckets);
+ Assert.assertArrayEquals(expected_counts, histogram);
+ }
+
+ @Test
public void map() {
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
JavaDoubleRDD doubles = rdd.map(new DoubleFunction<Integer>() {
@@ -837,7 +851,7 @@ public class JavaAPISuite implements Serializable {
public void checkpointAndComputation() {
File tempDir = Files.createTempDir();
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
- sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
+ sc.setCheckpointDir(tempDir.getAbsolutePath());
Assert.assertEquals(false, rdd.isCheckpointed());
rdd.checkpoint();
rdd.count(); // Forces the DAG to cause a checkpoint
@@ -849,7 +863,7 @@ public class JavaAPISuite implements Serializable {
public void checkpointAndRestore() {
File tempDir = Files.createTempDir();
JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5));
- sc.setCheckpointDir(tempDir.getAbsolutePath(), true);
+ sc.setCheckpointDir(tempDir.getAbsolutePath());
Assert.assertEquals(false, rdd.isCheckpointed());
rdd.checkpoint();
rdd.count(); // Forces the DAG to cause a checkpoint
@@ -883,4 +897,69 @@ public class JavaAPISuite implements Serializable {
new Tuple2<Integer, Integer>(0, 4)), rdd3.collect());
}
+
+ @Test
+ public void collectPartitions() {
+ JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3);
+
+ JavaPairRDD<Integer, Integer> rdd2 = rdd1.map(new PairFunction<Integer, Integer, Integer>() {
+ @Override
+ public Tuple2<Integer, Integer> call(Integer i) throws Exception {
+ return new Tuple2<Integer, Integer>(i, i % 2);
+ }
+ });
+
+ List[] parts = rdd1.collectPartitions(new int[] {0});
+ Assert.assertEquals(Arrays.asList(1, 2), parts[0]);
+
+ parts = rdd1.collectPartitions(new int[] {1, 2});
+ Assert.assertEquals(Arrays.asList(3, 4), parts[0]);
+ Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]);
+
+ Assert.assertEquals(Arrays.asList(new Tuple2<Integer, Integer>(1, 1),
+ new Tuple2<Integer, Integer>(2, 0)),
+ rdd2.collectPartitions(new int[] {0})[0]);
+
+ parts = rdd2.collectPartitions(new int[] {1, 2});
+ Assert.assertEquals(Arrays.asList(new Tuple2<Integer, Integer>(3, 1),
+ new Tuple2<Integer, Integer>(4, 0)),
+ parts[0]);
+ Assert.assertEquals(Arrays.asList(new Tuple2<Integer, Integer>(5, 1),
+ new Tuple2<Integer, Integer>(6, 0),
+ new Tuple2<Integer, Integer>(7, 1)),
+ parts[1]);
+ }
+
+ @Test
+ public void countApproxDistinct() {
+ List<Integer> arrayData = new ArrayList<Integer>();
+ int size = 100;
+ for (int i = 0; i < 100000; i++) {
+ arrayData.add(i % size);
+ }
+ JavaRDD<Integer> simpleRdd = sc.parallelize(arrayData, 10);
+ Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.2) - size) / (size * 1.0)) < 0.2);
+ Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.05);
+ Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.01) - size) / (size * 1.0)) <= 0.01);
+ }
+
+ @Test
+ public void countApproxDistinctByKey() {
+ double relativeSD = 0.001;
+
+ List<Tuple2<Integer, Integer>> arrayData = new ArrayList<Tuple2<Integer, Integer>>();
+ for (int i = 10; i < 100; i++)
+ for (int j = 0; j < i; j++)
+ arrayData.add(new Tuple2<Integer, Integer>(i, j));
+
+ JavaPairRDD<Integer, Integer> pairRdd = sc.parallelizePairs(arrayData);
+ List<Tuple2<Integer, Object>> res = pairRdd.countApproxDistinctByKey(relativeSD).collect();
+ for (Tuple2<Integer, Object> resItem : res) {
+ double count = (double)resItem._1();
+ Long resCount = (Long)resItem._2();
+ Double error = Math.abs((resCount - count) / count);
+ Assert.assertTrue(error < relativeSD);
+ }
+
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
index d8a0e983b2..1121e06e2e 100644
--- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala
@@ -114,7 +114,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf
// Once A is cancelled, job B should finish fairly quickly.
assert(jobB.get() === 100)
}
-
+/*
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
@@ -148,7 +148,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf
intercept[SparkException] { f1.get() }
intercept[SparkException] { f2.get() }
}
-
+ */
def testCount() {
// Cancel before launching any tasks
{
diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
index b7eb268bd5..afc1beff98 100644
--- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
@@ -23,9 +23,10 @@ import akka.actor._
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.storage.BlockManagerId
import org.apache.spark.util.AkkaUtils
+import scala.concurrent.Await
class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
-
+ private val conf = new SparkConf
test("compressSize") {
assert(MapOutputTracker.compressSize(0L) === 0)
assert(MapOutputTracker.compressSize(1L) === 1)
@@ -48,14 +49,14 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
test("master start and stop") {
val actorSystem = ActorSystem("test")
- val tracker = new MapOutputTrackerMaster()
+ val tracker = new MapOutputTrackerMaster(conf)
tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
tracker.stop()
}
test("master register and fetch") {
val actorSystem = ActorSystem("test")
- val tracker = new MapOutputTrackerMaster()
+ val tracker = new MapOutputTrackerMaster(conf)
tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
tracker.registerShuffle(10, 2)
val compressedSize1000 = MapOutputTracker.compressSize(1000L)
@@ -74,7 +75,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
test("master register and unregister and fetch") {
val actorSystem = ActorSystem("test")
- val tracker = new MapOutputTrackerMaster()
+ val tracker = new MapOutputTrackerMaster(conf)
tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker)))
tracker.registerShuffle(10, 2)
val compressedSize1000 = MapOutputTracker.compressSize(1000L)
@@ -96,18 +97,20 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext {
test("remote fetch") {
val hostname = "localhost"
- val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0)
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf)
System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext
System.setProperty("spark.hostPort", hostname + ":" + boundPort)
- val masterTracker = new MapOutputTrackerMaster()
+ val masterTracker = new MapOutputTrackerMaster(conf)
masterTracker.trackerActor = actorSystem.actorOf(
Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker")
- val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0)
- val slaveTracker = new MapOutputTracker()
- slaveTracker.trackerActor = slaveSystem.actorFor(
- "akka://spark@localhost:" + boundPort + "/user/MapOutputTracker")
+ val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf)
+ val slaveTracker = new MapOutputTracker(conf)
+ val selection = slaveSystem.actorSelection(
+ s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker")
+ val timeout = AkkaUtils.lookupTimeout(conf)
+ slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout)
masterTracker.registerShuffle(10, 1)
masterTracker.incrementEpoch()
diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
index 288aa14eeb..c650ef4ed5 100644
--- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
@@ -27,8 +27,10 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite =>
def sc: SparkContext = _sc
+ var conf = new SparkConf(false)
+
override def beforeAll() {
- _sc = new SparkContext("local", "test")
+ _sc = new SparkContext("local", "test", conf)
super.beforeAll()
}
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
new file mode 100644
index 0000000000..ef5936dd2f
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -0,0 +1,110 @@
+package org.apache.spark
+
+import org.scalatest.FunSuite
+
+class SparkConfSuite extends FunSuite with LocalSparkContext {
+ // This test uses the spark.conf in core/src/test/resources, which has a few test properties
+ test("loading from spark.conf") {
+ val conf = new SparkConf()
+ assert(conf.get("spark.test.intTestProperty") === "1")
+ assert(conf.get("spark.test.stringTestProperty") === "hi")
+ // NOTE: we don't use list properties yet, but when we do, we'll have to deal with this syntax
+ assert(conf.get("spark.test.listTestProperty") === "[a, b]")
+ }
+
+ // This test uses the spark.conf in core/src/test/resources, which has a few test properties
+ test("system properties override spark.conf") {
+ try {
+ System.setProperty("spark.test.intTestProperty", "2")
+ val conf = new SparkConf()
+ assert(conf.get("spark.test.intTestProperty") === "2")
+ assert(conf.get("spark.test.stringTestProperty") === "hi")
+ } finally {
+ System.clearProperty("spark.test.intTestProperty")
+ }
+ }
+
+ test("initializing without loading defaults") {
+ try {
+ System.setProperty("spark.test.intTestProperty", "2")
+ val conf = new SparkConf(false)
+ assert(!conf.contains("spark.test.intTestProperty"))
+ assert(!conf.contains("spark.test.stringTestProperty"))
+ } finally {
+ System.clearProperty("spark.test.intTestProperty")
+ }
+ }
+
+ test("named set methods") {
+ val conf = new SparkConf(false)
+
+ conf.setMaster("local[3]")
+ conf.setAppName("My app")
+ conf.setSparkHome("/path")
+ conf.setJars(Seq("a.jar", "b.jar"))
+ conf.setExecutorEnv("VAR1", "value1")
+ conf.setExecutorEnv(Seq(("VAR2", "value2"), ("VAR3", "value3")))
+
+ assert(conf.get("spark.master") === "local[3]")
+ assert(conf.get("spark.app.name") === "My app")
+ assert(conf.get("spark.home") === "/path")
+ assert(conf.get("spark.jars") === "a.jar,b.jar")
+ assert(conf.get("spark.executorEnv.VAR1") === "value1")
+ assert(conf.get("spark.executorEnv.VAR2") === "value2")
+ assert(conf.get("spark.executorEnv.VAR3") === "value3")
+
+ // Test the Java-friendly versions of these too
+ conf.setJars(Array("c.jar", "d.jar"))
+ conf.setExecutorEnv(Array(("VAR4", "value4"), ("VAR5", "value5")))
+ assert(conf.get("spark.jars") === "c.jar,d.jar")
+ assert(conf.get("spark.executorEnv.VAR4") === "value4")
+ assert(conf.get("spark.executorEnv.VAR5") === "value5")
+ }
+
+ test("basic get and set") {
+ val conf = new SparkConf(false)
+ assert(conf.getAll.toSet === Set())
+ conf.set("k1", "v1")
+ conf.setAll(Seq(("k2", "v2"), ("k3", "v3")))
+ assert(conf.getAll.toSet === Set(("k1", "v1"), ("k2", "v2"), ("k3", "v3")))
+ conf.set("k1", "v4")
+ conf.setAll(Seq(("k2", "v5"), ("k3", "v6")))
+ assert(conf.getAll.toSet === Set(("k1", "v4"), ("k2", "v5"), ("k3", "v6")))
+ assert(conf.contains("k1"), "conf did not contain k1")
+ assert(!conf.contains("k4"), "conf contained k4")
+ assert(conf.get("k1") === "v4")
+ intercept[Exception] { conf.get("k4") }
+ assert(conf.get("k4", "not found") === "not found")
+ assert(conf.getOption("k1") === Some("v4"))
+ assert(conf.getOption("k4") === None)
+ }
+
+ test("creating SparkContext without master and app name") {
+ val conf = new SparkConf(false)
+ intercept[SparkException] { sc = new SparkContext(conf) }
+ }
+
+ test("creating SparkContext without master") {
+ val conf = new SparkConf(false).setAppName("My app")
+ intercept[SparkException] { sc = new SparkContext(conf) }
+ }
+
+ test("creating SparkContext without app name") {
+ val conf = new SparkConf(false).setMaster("local")
+ intercept[SparkException] { sc = new SparkContext(conf) }
+ }
+
+ test("creating SparkContext with both master and app name") {
+ val conf = new SparkConf(false).setMaster("local").setAppName("My app")
+ sc = new SparkContext(conf)
+ assert(sc.master === "local")
+ assert(sc.appName === "My app")
+ }
+
+ test("SparkContext property overriding") {
+ val conf = new SparkConf(false).setMaster("local").setAppName("My app")
+ sc = new SparkContext("local[2]", "My other app", conf)
+ assert(sc.master === "local[2]")
+ assert(sc.appName === "My other app")
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
new file mode 100644
index 0000000000..f28d5c7b13
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala
@@ -0,0 +1,132 @@
+/*
+ * 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 org.scalatest.{FunSuite, PrivateMethodTester}
+
+import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskScheduler}
+import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend}
+import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
+import org.apache.spark.scheduler.local.LocalBackend
+
+class SparkContextSchedulerCreationSuite
+ extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging {
+
+ def createTaskScheduler(master: String): TaskSchedulerImpl = {
+ // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the
+ // real schedulers, so we don't want to create a full SparkContext with the desired scheduler.
+ sc = new SparkContext("local", "test")
+ val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler)
+ val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, "test")
+ sched.asInstanceOf[TaskSchedulerImpl]
+ }
+
+ test("bad-master") {
+ val e = intercept[SparkException] {
+ createTaskScheduler("localhost:1234")
+ }
+ assert(e.getMessage.contains("Could not parse Master URL"))
+ }
+
+ test("local") {
+ val sched = createTaskScheduler("local")
+ sched.backend match {
+ case s: LocalBackend => assert(s.totalCores === 1)
+ case _ => fail()
+ }
+ }
+
+ test("local-n") {
+ val sched = createTaskScheduler("local[5]")
+ assert(sched.maxTaskFailures === 1)
+ sched.backend match {
+ case s: LocalBackend => assert(s.totalCores === 5)
+ case _ => fail()
+ }
+ }
+
+ test("local-n-failures") {
+ val sched = createTaskScheduler("local[4, 2]")
+ assert(sched.maxTaskFailures === 2)
+ sched.backend match {
+ case s: LocalBackend => assert(s.totalCores === 4)
+ case _ => fail()
+ }
+ }
+
+ test("simr") {
+ createTaskScheduler("simr://uri").backend match {
+ case s: SimrSchedulerBackend => // OK
+ case _ => fail()
+ }
+ }
+
+ test("local-cluster") {
+ createTaskScheduler("local-cluster[3, 14, 512]").backend match {
+ case s: SparkDeploySchedulerBackend => // OK
+ case _ => fail()
+ }
+ }
+
+ def testYarn(master: String, expectedClassName: String) {
+ try {
+ val sched = createTaskScheduler(master)
+ assert(sched.getClass === Class.forName(expectedClassName))
+ } catch {
+ case e: SparkException =>
+ assert(e.getMessage.contains("YARN mode not available"))
+ logWarning("YARN not available, could not test actual YARN scheduler creation")
+ case e: Throwable => fail(e)
+ }
+ }
+
+ test("yarn-standalone") {
+ testYarn("yarn-standalone", "org.apache.spark.scheduler.cluster.YarnClusterScheduler")
+ }
+
+ test("yarn-client") {
+ testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler")
+ }
+
+ def testMesos(master: String, expectedClass: Class[_]) {
+ try {
+ val sched = createTaskScheduler(master)
+ assert(sched.backend.getClass === expectedClass)
+ } catch {
+ case e: UnsatisfiedLinkError =>
+ assert(e.getMessage.contains("no mesos in"))
+ logWarning("Mesos not available, could not test actual Mesos scheduler creation")
+ case e: Throwable => fail(e)
+ }
+ }
+
+ test("mesos fine-grained") {
+ System.setProperty("spark.mesos.coarse", "false")
+ testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend])
+ }
+
+ test("mesos coarse-grained") {
+ System.setProperty("spark.mesos.coarse", "true")
+ testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend])
+ }
+
+ test("mesos with zookeeper") {
+ System.setProperty("spark.mesos.coarse", "false")
+ testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend])
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
index 46a2da1724..768ca3850e 100644
--- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala
@@ -37,7 +37,7 @@ class UnpersistSuite extends FunSuite with LocalSparkContext {
Thread.sleep(200)
}
} catch {
- case _ => { Thread.sleep(10) }
+ case _: Throwable => { Thread.sleep(10) }
// Do nothing. We might see exceptions because block manager
// is racing this thread to remove entries from the driver.
}
diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
index 0b38e239f9..331fa3a642 100644
--- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
@@ -70,10 +70,11 @@ class JsonProtocolSuite extends FunSuite {
def createAppDesc() : ApplicationDescription = {
val cmd = new Command("mainClass", List("arg1", "arg2"), Map())
- new ApplicationDescription("name", 4, 1234, cmd, "sparkHome", "appUiUrl")
+ new ApplicationDescription("name", Some(4), 1234, cmd, "sparkHome", "appUiUrl")
}
def createAppInfo() : ApplicationInfo = {
- new ApplicationInfo(3, "id", createAppDesc(), new Date(123456789), null, "appUriStr")
+ new ApplicationInfo(
+ 3, "id", createAppDesc(), new Date(123456789), null, "appUriStr", Int.MaxValue)
}
def createWorkerInfo() : WorkerInfo = {
new WorkerInfo("id", "host", 8080, 4, 1234, null, 80, "publicAddress")
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
index 8f0954122b..be93074b7b 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala
@@ -1,14 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.deploy.worker
import java.io.File
+
import org.scalatest.FunSuite
+
import org.apache.spark.deploy.{ExecutorState, Command, ApplicationDescription}
class ExecutorRunnerTest extends FunSuite {
test("command includes appId") {
def f(s:String) = new File(s)
- val sparkHome = sys.env("SPARK_HOME")
- val appDesc = new ApplicationDescription("app name", 8, 500, Command("foo", Seq(),Map()),
+ val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get
+ val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()),
sparkHome, "appUiUrl")
val appId = "12345-worker321-9876"
val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome),
diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
index ab81bfbe55..8d7546085f 100644
--- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
+++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala
@@ -20,9 +20,11 @@ package org.apache.spark.io
import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
import org.scalatest.FunSuite
+import org.apache.spark.SparkConf
class CompressionCodecSuite extends FunSuite {
+ val conf = new SparkConf(false)
def testCodec(codec: CompressionCodec) {
// Write 1000 integers to the output stream, compressed.
@@ -43,19 +45,19 @@ class CompressionCodecSuite extends FunSuite {
}
test("default compression codec") {
- val codec = CompressionCodec.createCodec()
+ val codec = CompressionCodec.createCodec(conf)
assert(codec.getClass === classOf[LZFCompressionCodec])
testCodec(codec)
}
test("lzf compression codec") {
- val codec = CompressionCodec.createCodec(classOf[LZFCompressionCodec].getName)
+ val codec = CompressionCodec.createCodec(conf, classOf[LZFCompressionCodec].getName)
assert(codec.getClass === classOf[LZFCompressionCodec])
testCodec(codec)
}
test("snappy compression codec") {
- val codec = CompressionCodec.createCodec(classOf[SnappyCompressionCodec].getName)
+ val codec = CompressionCodec.createCodec(conf, classOf[SnappyCompressionCodec].getName)
assert(codec.getClass === classOf[SnappyCompressionCodec])
testCodec(codec)
}
diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
index 7181333adf..71a2c6c498 100644
--- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
+++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala
@@ -19,17 +19,19 @@ package org.apache.spark.metrics
import org.scalatest.{BeforeAndAfter, FunSuite}
import org.apache.spark.deploy.master.MasterSource
+import org.apache.spark.SparkConf
class MetricsSystemSuite extends FunSuite with BeforeAndAfter {
var filePath: String = _
+ var conf: SparkConf = null
before {
filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile()
- System.setProperty("spark.metrics.conf", filePath)
+ conf = new SparkConf(false).set("spark.metrics.conf", filePath)
}
test("MetricsSystem with default config") {
- val metricsSystem = MetricsSystem.createMetricsSystem("default")
+ val metricsSystem = MetricsSystem.createMetricsSystem("default", conf)
val sources = metricsSystem.sources
val sinks = metricsSystem.sinks
@@ -39,7 +41,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter {
}
test("MetricsSystem with sources add") {
- val metricsSystem = MetricsSystem.createMetricsSystem("test")
+ val metricsSystem = MetricsSystem.createMetricsSystem("test", conf)
val sources = metricsSystem.sources
val sinks = metricsSystem.sinks
diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
index da032b17d9..0d4c10db8e 100644
--- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala
@@ -19,6 +19,8 @@ package org.apache.spark.rdd
import java.util.concurrent.Semaphore
+import scala.concurrent.{Await, TimeoutException}
+import scala.concurrent.duration.Duration
import scala.concurrent.ExecutionContext.Implicits.global
import org.scalatest.{BeforeAndAfterAll, FunSuite}
@@ -173,4 +175,28 @@ class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts
sem.acquire(2)
}
}
+
+ /**
+ * Awaiting FutureAction results
+ */
+ test("FutureAction result, infinite wait") {
+ val f = sc.parallelize(1 to 100, 4)
+ .countAsync()
+ assert(Await.result(f, Duration.Inf) === 100)
+ }
+
+ test("FutureAction result, finite wait") {
+ val f = sc.parallelize(1 to 100, 4)
+ .countAsync()
+ assert(Await.result(f, Duration(30, "seconds")) === 100)
+ }
+
+ test("FutureAction result, timeout") {
+ val f = sc.parallelize(1 to 100, 4)
+ .mapPartitions(itr => { Thread.sleep(20); itr })
+ .countAsync()
+ intercept[TimeoutException] {
+ Await.result(f, Duration(20, "milliseconds"))
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala
new file mode 100644
index 0000000000..7f50a5a47c
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala
@@ -0,0 +1,271 @@
+/*
+ * 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 scala.math.abs
+import scala.collection.mutable.ArrayBuffer
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.SparkContext._
+import org.apache.spark.rdd._
+import org.apache.spark._
+
+class DoubleRDDSuite extends FunSuite with SharedSparkContext {
+ // Verify tests on the histogram functionality. We test with both evenly
+ // and non-evenly spaced buckets as the bucket lookup function changes.
+ test("WorksOnEmpty") {
+ // Make sure that it works on an empty input
+ val rdd: RDD[Double] = sc.parallelize(Seq())
+ val buckets = Array(0.0, 10.0)
+ val histogramResults = rdd.histogram(buckets)
+ val histogramResults2 = rdd.histogram(buckets, true)
+ val expectedHistogramResults = Array(0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramResults2 === expectedHistogramResults)
+ }
+
+ test("WorksWithOutOfRangeWithOneBucket") {
+ // Verify that if all of the elements are out of range the counts are zero
+ val rdd = sc.parallelize(Seq(10.01, -0.01))
+ val buckets = Array(0.0, 10.0)
+ val histogramResults = rdd.histogram(buckets)
+ val histogramResults2 = rdd.histogram(buckets, true)
+ val expectedHistogramResults = Array(0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramResults2 === expectedHistogramResults)
+ }
+
+ test("WorksInRangeWithOneBucket") {
+ // Verify the basic case of one bucket and all elements in that bucket works
+ val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+ val buckets = Array(0.0, 10.0)
+ val histogramResults = rdd.histogram(buckets)
+ val histogramResults2 = rdd.histogram(buckets, true)
+ val expectedHistogramResults = Array(4)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramResults2 === expectedHistogramResults)
+ }
+
+ test("WorksInRangeWithOneBucketExactMatch") {
+ // Verify the basic case of one bucket and all elements in that bucket works
+ val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+ val buckets = Array(1.0, 4.0)
+ val histogramResults = rdd.histogram(buckets)
+ val histogramResults2 = rdd.histogram(buckets, true)
+ val expectedHistogramResults = Array(4)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramResults2 === expectedHistogramResults)
+ }
+
+ test("WorksWithOutOfRangeWithTwoBuckets") {
+ // Verify that out of range works with two buckets
+ val rdd = sc.parallelize(Seq(10.01, -0.01))
+ val buckets = Array(0.0, 5.0, 10.0)
+ val histogramResults = rdd.histogram(buckets)
+ val histogramResults2 = rdd.histogram(buckets, true)
+ val expectedHistogramResults = Array(0, 0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramResults2 === expectedHistogramResults)
+ }
+
+ test("WorksWithOutOfRangeWithTwoUnEvenBuckets") {
+ // Verify that out of range works with two un even buckets
+ val rdd = sc.parallelize(Seq(10.01, -0.01))
+ val buckets = Array(0.0, 4.0, 10.0)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(0, 0)
+ assert(histogramResults === expectedHistogramResults)
+ }
+
+ test("WorksInRangeWithTwoBuckets") {
+ // Make sure that it works with two equally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6))
+ val buckets = Array(0.0, 5.0, 10.0)
+ val histogramResults = rdd.histogram(buckets)
+ val histogramResults2 = rdd.histogram(buckets, true)
+ val expectedHistogramResults = Array(3, 2)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramResults2 === expectedHistogramResults)
+ }
+
+ test("WorksInRangeWithTwoBucketsAndNaN") {
+ // Make sure that it works with two equally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6, Double.NaN))
+ val buckets = Array(0.0, 5.0, 10.0)
+ val histogramResults = rdd.histogram(buckets)
+ val histogramResults2 = rdd.histogram(buckets, true)
+ val expectedHistogramResults = Array(3, 2)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramResults2 === expectedHistogramResults)
+ }
+
+ test("WorksInRangeWithTwoUnevenBuckets") {
+ // Make sure that it works with two unequally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(1, 2, 3, 5, 6))
+ val buckets = Array(0.0, 5.0, 11.0)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(3, 2)
+ assert(histogramResults === expectedHistogramResults)
+ }
+
+ test("WorksMixedRangeWithTwoUnevenBuckets") {
+ // Make sure that it works with two unequally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01))
+ val buckets = Array(0.0, 5.0, 11.0)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(4, 3)
+ assert(histogramResults === expectedHistogramResults)
+ }
+
+ test("WorksMixedRangeWithFourUnevenBuckets") {
+ // Make sure that it works with two unequally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+ 200.0, 200.1))
+ val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(4, 2, 1, 3)
+ assert(histogramResults === expectedHistogramResults)
+ }
+
+ test("WorksMixedRangeWithUnevenBucketsAndNaN") {
+ // Make sure that it works with two unequally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+ 200.0, 200.1, Double.NaN))
+ val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(4, 2, 1, 3)
+ assert(histogramResults === expectedHistogramResults)
+ }
+ // Make sure this works with a NaN end bucket
+ test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRange") {
+ // Make sure that it works with two unequally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+ 200.0, 200.1, Double.NaN))
+ val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(4, 2, 1, 2, 3)
+ assert(histogramResults === expectedHistogramResults)
+ }
+ // Make sure this works with a NaN end bucket and an inifity
+ test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfity") {
+ // Make sure that it works with two unequally spaced buckets and elements in each
+ val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0,
+ 200.0, 200.1, 1.0/0.0, -1.0/0.0, Double.NaN))
+ val buckets = Array(0.0, 5.0, 11.0, 12.0, 200.0, Double.NaN)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(4, 2, 1, 2, 4)
+ assert(histogramResults === expectedHistogramResults)
+ }
+
+ test("WorksWithOutOfRangeWithInfiniteBuckets") {
+ // Verify that out of range works with two buckets
+ val rdd = sc.parallelize(Seq(10.01, -0.01, Double.NaN))
+ val buckets = Array(-1.0/0.0 , 0.0, 1.0/0.0)
+ val histogramResults = rdd.histogram(buckets)
+ val expectedHistogramResults = Array(1, 1)
+ assert(histogramResults === expectedHistogramResults)
+ }
+ // Test the failure mode with an invalid bucket array
+ test("ThrowsExceptionOnInvalidBucketArray") {
+ val rdd = sc.parallelize(Seq(1.0))
+ // Empty array
+ intercept[IllegalArgumentException] {
+ val buckets = Array.empty[Double]
+ val result = rdd.histogram(buckets)
+ }
+ // Single element array
+ intercept[IllegalArgumentException] {
+ val buckets = Array(1.0)
+ val result = rdd.histogram(buckets)
+ }
+ }
+
+ // Test automatic histogram function
+ test("WorksWithoutBucketsBasic") {
+ // Verify the basic case of one bucket and all elements in that bucket works
+ val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+ val (histogramBuckets, histogramResults) = rdd.histogram(1)
+ val expectedHistogramResults = Array(4)
+ val expectedHistogramBuckets = Array(1.0, 4.0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramBuckets === expectedHistogramBuckets)
+ }
+ // Test automatic histogram function with a single element
+ test("WorksWithoutBucketsBasicSingleElement") {
+ // Verify the basic case of one bucket and all elements in that bucket works
+ val rdd = sc.parallelize(Seq(1))
+ val (histogramBuckets, histogramResults) = rdd.histogram(1)
+ val expectedHistogramResults = Array(1)
+ val expectedHistogramBuckets = Array(1.0, 1.0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramBuckets === expectedHistogramBuckets)
+ }
+ // Test automatic histogram function with a single element
+ test("WorksWithoutBucketsBasicNoRange") {
+ // Verify the basic case of one bucket and all elements in that bucket works
+ val rdd = sc.parallelize(Seq(1, 1, 1, 1))
+ val (histogramBuckets, histogramResults) = rdd.histogram(1)
+ val expectedHistogramResults = Array(4)
+ val expectedHistogramBuckets = Array(1.0, 1.0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramBuckets === expectedHistogramBuckets)
+ }
+
+ test("WorksWithoutBucketsBasicTwo") {
+ // Verify the basic case of one bucket and all elements in that bucket works
+ val rdd = sc.parallelize(Seq(1, 2, 3, 4))
+ val (histogramBuckets, histogramResults) = rdd.histogram(2)
+ val expectedHistogramResults = Array(2, 2)
+ val expectedHistogramBuckets = Array(1.0, 2.5, 4.0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramBuckets === expectedHistogramBuckets)
+ }
+
+ test("WorksWithoutBucketsWithMoreRequestedThanElements") {
+ // Verify the basic case of one bucket and all elements in that bucket works
+ val rdd = sc.parallelize(Seq(1, 2))
+ val (histogramBuckets, histogramResults) = rdd.histogram(10)
+ val expectedHistogramResults =
+ Array(1, 0, 0, 0, 0, 0, 0, 0, 0, 1)
+ val expectedHistogramBuckets =
+ Array(1.0, 1.1, 1.2, 1.3, 1.4, 1.5, 1.6, 1.7, 1.8, 1.9, 2.0)
+ assert(histogramResults === expectedHistogramResults)
+ assert(histogramBuckets === expectedHistogramBuckets)
+ }
+
+ // Test the failure mode with an invalid RDD
+ test("ThrowsExceptionOnInvalidRDDs") {
+ // infinity
+ intercept[UnsupportedOperationException] {
+ val rdd = sc.parallelize(Seq(1, 1.0/0.0))
+ val result = rdd.histogram(1)
+ }
+ // NaN
+ intercept[UnsupportedOperationException] {
+ val rdd = sc.parallelize(Seq(1, Double.NaN))
+ val result = rdd.histogram(1)
+ }
+ // Empty
+ intercept[UnsupportedOperationException] {
+ val rdd: RDD[Double] = sc.parallelize(Seq())
+ val result = rdd.histogram(1)
+ }
+ }
+
+}
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 57d3382ed0..5da538a1dd 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.rdd
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashSet
+import scala.util.Random
import org.scalatest.FunSuite
@@ -109,6 +110,39 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext {
assert(deps.size === 2) // ShuffledRDD, ParallelCollection.
}
+ test("countApproxDistinctByKey") {
+ def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble
+
+ /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is
+ * only a statistical bound, the tests can fail for large values of relativeSD. We will be using
+ * relatively tight error bounds to check correctness of functionality rather than checking
+ * whether the approximation conforms with the requested bound.
+ */
+ val relativeSD = 0.001
+
+ // For each value i, there are i tuples with first element equal to i.
+ // Therefore, the expected count for key i would be i.
+ val stacked = (1 to 100).flatMap(i => (1 to i).map(j => (i, j)))
+ val rdd1 = sc.parallelize(stacked)
+ val counted1 = rdd1.countApproxDistinctByKey(relativeSD).collect()
+ counted1.foreach{
+ case(k, count) => assert(error(count, k) < relativeSD)
+ }
+
+ val rnd = new Random()
+
+ // The expected count for key num would be num
+ val randStacked = (1 to 100).flatMap { i =>
+ val num = rnd.nextInt % 500
+ (1 to num).map(j => (num, j))
+ }
+ val rdd2 = sc.parallelize(randStacked)
+ val counted2 = rdd2.countApproxDistinctByKey(relativeSD, 4).collect()
+ counted2.foreach{
+ case(k, count) => assert(error(count, k) < relativeSD)
+ }
+ }
+
test("join") {
val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1)))
val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w')))
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 354ab8ae5d..559ea051d3 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -63,6 +63,19 @@ class RDDSuite extends FunSuite with SharedSparkContext {
}
}
+ test("countApproxDistinct") {
+
+ def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble
+
+ val size = 100
+ val uniformDistro = for (i <- 1 to 100000) yield i % size
+ val simpleRdd = sc.makeRDD(uniformDistro)
+ assert(error(simpleRdd.countApproxDistinct(0.2), size) < 0.2)
+ assert(error(simpleRdd.countApproxDistinct(0.05), size) < 0.05)
+ assert(error(simpleRdd.countApproxDistinct(0.01), size) < 0.01)
+ assert(error(simpleRdd.countApproxDistinct(0.001), size) < 0.001)
+ }
+
test("SparkContext.union") {
val nums = sc.makeRDD(Array(1, 2, 3, 4), 2)
assert(sc.union(nums).collect().toList === List(1, 2, 3, 4))
@@ -71,6 +84,33 @@ class RDDSuite extends FunSuite with SharedSparkContext {
assert(sc.union(Seq(nums, nums)).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4))
}
+ test("partitioner aware union") {
+ import SparkContext._
+ def makeRDDWithPartitioner(seq: Seq[Int]) = {
+ sc.makeRDD(seq, 1)
+ .map(x => (x, null))
+ .partitionBy(new HashPartitioner(2))
+ .mapPartitions(_.map(_._1), true)
+ }
+
+ val nums1 = makeRDDWithPartitioner(1 to 4)
+ val nums2 = makeRDDWithPartitioner(5 to 8)
+ assert(nums1.partitioner == nums2.partitioner)
+ assert(new PartitionerAwareUnionRDD(sc, Seq(nums1)).collect().toSet === Set(1, 2, 3, 4))
+
+ val union = new PartitionerAwareUnionRDD(sc, Seq(nums1, nums2))
+ assert(union.collect().toSet === Set(1, 2, 3, 4, 5, 6, 7, 8))
+ val nums1Parts = nums1.collectPartitions()
+ val nums2Parts = nums2.collectPartitions()
+ val unionParts = union.collectPartitions()
+ assert(nums1Parts.length === 2)
+ assert(nums2Parts.length === 2)
+ assert(unionParts.length === 2)
+ assert((nums1Parts(0) ++ nums2Parts(0)).toList === unionParts(0).toList)
+ assert((nums1Parts(1) ++ nums2Parts(1)).toList === unionParts(1).toList)
+ assert(union.partitioner === nums1.partitioner)
+ }
+
test("aggregate") {
val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3)))
type StringMap = HashMap[String, Int]
@@ -244,8 +284,8 @@ class RDDSuite extends FunSuite with SharedSparkContext {
// test that you get over 90% locality in each group
val minLocality = coalesced2.partitions
.map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction)
- .foldLeft(1.)((perc, loc) => math.min(perc,loc))
- assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.).toInt + "%")
+ .foldLeft(1.0)((perc, loc) => math.min(perc,loc))
+ assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%")
// test that the groups are load balanced with 100 +/- 20 elements in each
val maxImbalance = coalesced2.partitions
@@ -257,9 +297,9 @@ class RDDSuite extends FunSuite with SharedSparkContext {
val coalesced3 = data3.coalesce(numMachines*2)
val minLocality2 = coalesced3.partitions
.map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction)
- .foldLeft(1.)((perc, loc) => math.min(perc,loc))
+ .foldLeft(1.0)((perc, loc) => math.min(perc,loc))
assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " +
- (minLocality2*100.).toInt + "%")
+ (minLocality2*100.0).toInt + "%")
}
test("zipped RDDs") {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
index 95d3553d91..7bf2020fe3 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala
@@ -15,14 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
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 java.util.Properties
@@ -31,9 +29,9 @@ class FakeTaskSetManager(
initPriority: Int,
initStageId: Int,
initNumTasks: Int,
- clusterScheduler: ClusterScheduler,
+ clusterScheduler: TaskSchedulerImpl,
taskSet: TaskSet)
- extends ClusterTaskSetManager(clusterScheduler, taskSet) {
+ extends TaskSetManager(clusterScheduler, taskSet, 0) {
parent = null
weight = 1
@@ -106,7 +104,7 @@ class FakeTaskSetManager(
class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging {
- def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: ClusterScheduler, taskSet: TaskSet): FakeTaskSetManager = {
+ def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, taskSet: TaskSet): FakeTaskSetManager = {
new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet)
}
@@ -133,7 +131,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
test("FIFO Scheduler Test") {
sc = new SparkContext("local", "ClusterSchedulerSuite")
- val clusterScheduler = new ClusterScheduler(sc)
+ val clusterScheduler = new TaskSchedulerImpl(sc)
var tasks = ArrayBuffer[Task[_]]()
val task = new FakeTask(0)
tasks += task
@@ -160,7 +158,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
test("Fair Scheduler Test") {
sc = new SparkContext("local", "ClusterSchedulerSuite")
- val clusterScheduler = new ClusterScheduler(sc)
+ val clusterScheduler = new TaskSchedulerImpl(sc)
var tasks = ArrayBuffer[Task[_]]()
val task = new FakeTask(0)
tasks += task
@@ -169,7 +167,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile()
System.setProperty("spark.scheduler.allocation.file", xmlPath)
val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
- val schedulableBuilder = new FairSchedulableBuilder(rootPool)
+ val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf)
schedulableBuilder.buildPools()
assert(rootPool.getSchedulableByName("default") != null)
@@ -217,7 +215,7 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging
test("Nested Pool Test") {
sc = new SparkContext("local", "ClusterSchedulerSuite")
- val clusterScheduler = new ClusterScheduler(sc)
+ val clusterScheduler = new TaskSchedulerImpl(sc)
var tasks = ArrayBuffer[Task[_]]()
val task = new FakeTask(0)
tasks += task
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 a4d41ebbff..2aa259daf3 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
@@ -17,21 +17,14 @@
package org.apache.spark.scheduler
-import scala.collection.mutable.{Map, HashMap}
-
-import org.scalatest.FunSuite
-import org.scalatest.BeforeAndAfter
-
-import org.apache.spark.LocalSparkContext
-import org.apache.spark.MapOutputTrackerMaster
-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 scala.Tuple2
+import scala.collection.mutable.{HashMap, Map}
+
+import org.apache.spark._
import org.apache.spark.rdd.RDD
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
+import org.scalatest.{BeforeAndAfter, FunSuite}
/**
* Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler
@@ -46,7 +39,7 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
* and capturing the resulting TaskSets from the mock TaskScheduler.
*/
class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
-
+ val conf = new SparkConf
/** Set of TaskSets the DAGScheduler has requested executed. */
val taskSets = scala.collection.mutable.Buffer[TaskSet]()
val taskScheduler = new TaskScheduler() {
@@ -74,7 +67,7 @@ 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) {
+ val blockManagerMaster = new BlockManagerMaster(null, conf) {
override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = {
blockIds.map {
_.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)).
@@ -99,7 +92,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
taskSets.clear()
cacheLocations.clear()
results.clear()
- mapOutputTracker = new MapOutputTrackerMaster()
+ mapOutputTracker = new MapOutputTrackerMaster(conf)
scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) {
override def runLocally(job: ActiveJob) {
// don't bother with the thread while unit testing
@@ -206,6 +199,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
submit(rdd, Array(0))
complete(taskSets(0), List((Success, 42)))
assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
}
test("local job") {
@@ -219,6 +213,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
val jobId = scheduler.nextJobId.getAndIncrement()
runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener))
assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
}
test("run trivial job w/ dependency") {
@@ -227,6 +222,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
submit(finalRdd, Array(0))
complete(taskSets(0), Seq((Success, 42)))
assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
}
test("cache location preferences w/ dependency") {
@@ -239,12 +235,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
assertLocations(taskSet, Seq(Seq("hostA", "hostB")))
complete(taskSet, Seq((Success, 42)))
assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
}
test("trivial job failure") {
submit(makeRdd(1, Nil), Array(0))
failed(taskSets(0), "some failure")
assert(failure.getMessage === "Job aborted: some failure")
+ assertDataStructuresEmpty
}
test("run trivial shuffle") {
@@ -260,6 +258,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")))
complete(taskSets(1), Seq((Success, 42)))
assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
}
test("run trivial shuffle with fetch failure") {
@@ -285,6 +284,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB"))
complete(taskSets(3), Seq((Success, 43)))
assert(results === Map(0 -> 42, 1 -> 43))
+ assertDataStructuresEmpty
}
test("ignore late map task completions") {
@@ -313,6 +313,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA")))
complete(taskSets(1), Seq((Success, 42), (Success, 43)))
assert(results === Map(0 -> 42, 1 -> 43))
+ assertDataStructuresEmpty
}
test("run trivial shuffle with out-of-band failure and retry") {
@@ -329,15 +330,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
complete(taskSets(0), Seq(
(Success, makeMapStatus("hostA", 1)),
(Success, makeMapStatus("hostB", 1))))
- // have hostC complete the resubmitted task
- complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1))))
- assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
- Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB")))
- complete(taskSets(2), Seq((Success, 42)))
- assert(results === Map(0 -> 42))
- }
-
- test("recursive shuffle failures") {
+ // have hostC complete the resubmitted task
+ complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1))))
+ assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) ===
+ Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB")))
+ complete(taskSets(2), Seq((Success, 42)))
+ assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
+ }
+
+ test("recursive shuffle failures") {
val shuffleOneRdd = makeRdd(2, Nil)
val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null)
val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne))
@@ -363,6 +365,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1))))
complete(taskSets(5), Seq((Success, 42)))
assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
}
test("cached post-shuffle") {
@@ -394,6 +397,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1))))
complete(taskSets(4), Seq((Success, 42)))
assert(results === Map(0 -> 42))
+ assertDataStructuresEmpty
}
/**
@@ -413,4 +417,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont
private def makeBlockManagerId(host: String): BlockManagerId =
BlockManagerId("exec-" + host, host, 12345, 0)
+ private def assertDataStructuresEmpty = {
+ assert(scheduler.pendingTasks.isEmpty)
+ assert(scheduler.activeJobs.isEmpty)
+ assert(scheduler.failed.isEmpty)
+ assert(scheduler.idToActiveJob.isEmpty)
+ assert(scheduler.jobIdToStageIds.isEmpty)
+ assert(scheduler.stageIdToJobIds.isEmpty)
+ assert(scheduler.stageIdToStage.isEmpty)
+ assert(scheduler.stageToInfos.isEmpty)
+ assert(scheduler.resultStageToJob.isEmpty)
+ assert(scheduler.running.isEmpty)
+ assert(scheduler.shuffleToMapStage.isEmpty)
+ assert(scheduler.waiting.isEmpty)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
index 0f01515179..0b90c4e74c 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import org.apache.spark.TaskContext
-import org.apache.spark.scheduler.{TaskLocation, Task}
class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0) {
override def runTask(context: TaskContext): Int = 0
diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
index 984881861c..5cc48ee00a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala
@@ -31,6 +31,7 @@ import org.apache.spark.rdd.RDD
class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers {
+ val WAIT_TIMEOUT_MILLIS = 10000
test("inner method") {
sc = new SparkContext("local", "joblogger")
@@ -92,7 +93,9 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) }
rdd.reduceByKey(_+_).collect()
- val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER)
+ assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+
+ val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER)
joblogger.getLogDir should be ("/tmp/spark-%s".format(user))
joblogger.getJobIDtoPrintWriter.size should be (1)
@@ -114,13 +117,15 @@ class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1
override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1
override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1
- override def onStageCompleted(stageCompleted: StageCompleted) = onStageCompletedCount += 1
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = onStageCompletedCount += 1
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1
}
sc.addSparkListener(joblogger)
val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) }
rdd.reduceByKey(_+_).collect()
-
+
+ assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS))
+
joblogger.onJobStartCount should be (1)
joblogger.onJobEndCount should be (1)
joblogger.onTaskEndCount should be (8)
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 1fd76420ea..1a16e438c4 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -19,23 +19,26 @@ package org.apache.spark.scheduler
import scala.collection.mutable.{Buffer, HashSet}
-import org.scalatest.{BeforeAndAfterAll, FunSuite}
+import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite}
import org.scalatest.matchers.ShouldMatchers
import org.apache.spark.{LocalSparkContext, SparkContext}
import org.apache.spark.SparkContext._
class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers
- with BeforeAndAfterAll {
+ with BeforeAndAfter with BeforeAndAfterAll {
/** Length of time to wait while draining listener events. */
val WAIT_TIMEOUT_MILLIS = 10000
+ before {
+ sc = new SparkContext("local", "SparkListenerSuite")
+ }
+
override def afterAll {
System.clearProperty("spark.akka.frameSize")
}
test("basic creation of StageInfo") {
- sc = new SparkContext("local", "DAGSchedulerSuite")
val listener = new SaveStageInfo
sc.addSparkListener(listener)
val rdd1 = sc.parallelize(1 to 100, 4)
@@ -56,7 +59,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
}
test("StageInfo with fewer tasks than partitions") {
- sc = new SparkContext("local", "DAGSchedulerSuite")
val listener = new SaveStageInfo
sc.addSparkListener(listener)
val rdd1 = sc.parallelize(1 to 100, 4)
@@ -72,7 +74,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
}
test("local metrics") {
- sc = new SparkContext("local", "DAGSchedulerSuite")
val listener = new SaveStageInfo
sc.addSparkListener(listener)
sc.addSparkListener(new StatsReportListener)
@@ -135,17 +136,13 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
}
test("onTaskGettingResult() called when result fetched remotely") {
- // Need to use local cluster mode here, because results are not ever returned through the
- // block manager when using the LocalScheduler.
- sc = new SparkContext("local-cluster[1,1,512]", "test")
-
val listener = new SaveTaskEvents
sc.addSparkListener(listener)
// Make a task whose result is larger than the akka frame size
System.setProperty("spark.akka.frameSize", "1")
val akkaFrameSize =
- sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt
+ sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt
val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x,y) => x)
assert(result === 1.to(akkaFrameSize).toArray)
@@ -157,10 +154,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
}
test("onTaskGettingResult() not called when result sent directly") {
- // Need to use local cluster mode here, because results are not ever returned through the
- // block manager when using the LocalScheduler.
- sc = new SparkContext("local-cluster[1,1,512]", "test")
-
val listener = new SaveTaskEvents
sc.addSparkListener(listener)
@@ -181,7 +174,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc
class SaveStageInfo extends SparkListener {
val stageInfos = Buffer[StageInfo]()
- override def onStageCompleted(stage: StageCompleted) {
+ override def onStageCompleted(stage: SparkListenerStageCompleted) {
stageInfos += stage.stage
}
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
index ee150a3107..4b52d9651e 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
@@ -15,14 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
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.{LocalSparkContext, SparkConf, SparkContext, SparkEnv}
import org.apache.spark.storage.TaskResultBlockId
/**
@@ -31,12 +30,12 @@ import org.apache.spark.storage.TaskResultBlockId
* 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)
+class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedulerImpl)
extends TaskResultGetter(sparkEnv, scheduler) {
var removedResult = false
override def enqueueSuccessfulTask(
- taskSetManager: ClusterTaskSetManager, tid: Long, serializedData: ByteBuffer) {
+ taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) {
if (!removedResult) {
// Only remove the result once, since we'd like to test the case where the task eventually
// succeeds.
@@ -44,13 +43,13 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: ClusterSched
case IndirectTaskResult(blockId) =>
sparkEnv.blockManager.master.removeBlock(blockId)
case directResult: DirectTaskResult[_] =>
- taskSetManager.abort("Internal error: expect only indirect results")
+ taskSetManager.abort("Internal error: expect only indirect results")
}
serializedData.rewind()
removedResult = true
}
super.enqueueSuccessfulTask(taskSetManager, tid, serializedData)
- }
+ }
}
/**
@@ -65,24 +64,20 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA
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") {
+ sc = new SparkContext("local", "test")
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") {
+ test("handling results larger than Akka frame size") {
+ sc = new SparkContext("local", "test")
val akkaFrameSize =
- sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt
+ sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt
val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x)
assert(result === 1.to(akkaFrameSize).toArray)
@@ -92,10 +87,13 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA
}
test("task retried if result missing from block manager") {
+ // Set the maximum number of task failures to > 0, so that the task set isn't aborted
+ // after the result is missing.
+ sc = new SparkContext("local[1,2]", "test")
// 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 =>
+ val scheduler: TaskSchedulerImpl = sc.taskScheduler match {
+ case clusterScheduler: TaskSchedulerImpl =>
clusterScheduler
case _ =>
assert(false, "Expect local cluster to use ClusterScheduler")
@@ -103,7 +101,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA
}
scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler)
val akkaFrameSize =
- sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.message-frame-size").toInt
+ sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt
val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x)
assert(result === 1.to(akkaFrameSize).toArray)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index b97f2b19b5..1eec6726f4 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.scheduler.cluster
+package org.apache.spark.scheduler
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable
@@ -23,7 +23,6 @@ import scala.collection.mutable
import org.scalatest.FunSuite
import org.apache.spark._
-import org.apache.spark.scheduler._
import org.apache.spark.executor.TaskMetrics
import java.nio.ByteBuffer
import org.apache.spark.util.{Utils, FakeClock}
@@ -56,10 +55,10 @@ class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler
* A mock ClusterScheduler implementation that just remembers information about tasks started and
* feedback received from the TaskSetManagers. Note that it's important to initialize this with
* a list of "live" executors and their hostnames for isExecutorAlive and hasExecutorsAliveOnHost
- * to work, and these are required for locality in ClusterTaskSetManager.
+ * to work, and these are required for locality in TaskSetManager.
*/
class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */)
- extends ClusterScheduler(sc)
+ extends TaskSchedulerImpl(sc)
{
val startedTasks = new ArrayBuffer[Long]
val endedTasks = new mutable.HashMap[Long, TaskEndReason]
@@ -79,16 +78,19 @@ class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /*
override def hasExecutorsAliveOnHost(host: String): Boolean = executors.values.exists(_ == host)
}
-class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
+class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL}
- val LOCALITY_WAIT = System.getProperty("spark.locality.wait", "3000").toLong
+ private val conf = new SparkConf
+
+ val LOCALITY_WAIT = conf.get("spark.locality.wait", "3000").toLong
+ val MAX_TASK_FAILURES = 4
test("TaskSet with no preferences") {
sc = new SparkContext("local", "test")
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
val taskSet = createTaskSet(1)
- val manager = new ClusterTaskSetManager(sched, taskSet)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES)
// Offer a host with no CPUs
assert(manager.resourceOffer("exec1", "host1", 0, ANY) === None)
@@ -114,7 +116,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
sc = new SparkContext("local", "test")
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
val taskSet = createTaskSet(3)
- val manager = new ClusterTaskSetManager(sched, taskSet)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES)
// First three offers should all find tasks
for (i <- 0 until 3) {
@@ -151,7 +153,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
Seq() // Last task has no locality prefs
)
val clock = new FakeClock
- val manager = new ClusterTaskSetManager(sched, taskSet, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1, exec1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
@@ -197,7 +199,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
Seq(TaskLocation("host2"))
)
val clock = new FakeClock
- val manager = new ClusterTaskSetManager(sched, taskSet, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
@@ -234,7 +236,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
Seq(TaskLocation("host3"))
)
val clock = new FakeClock
- val manager = new ClusterTaskSetManager(sched, taskSet, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1: first task should be chosen
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
@@ -262,7 +264,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
val taskSet = createTaskSet(1)
val clock = new FakeClock
- val manager = new ClusterTaskSetManager(sched, taskSet, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
@@ -279,17 +281,17 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
val sched = new FakeClusterScheduler(sc, ("exec1", "host1"))
val taskSet = createTaskSet(1)
val clock = new FakeClock
- val manager = new ClusterTaskSetManager(sched, taskSet, clock)
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, 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 =>
+ (1 to manager.maxTaskFailures).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) {
+ if (index < MAX_TASK_FAILURES) {
assert(!sched.taskSetsFailed.contains(taskSet.id))
} else {
assert(sched.taskSetsFailed.contains(taskSet.id))
@@ -313,6 +315,7 @@ class ClusterTaskSetManagerSuite extends FunSuite with LocalSparkContext with Lo
}
def createTaskResult(id: Int): DirectTaskResult[Int] = {
- new DirectTaskResult[Int](id, mutable.Map.empty, new TaskMetrics)
+ val valueSer = SparkEnv.get.serializer.newInstance()
+ new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics)
}
}
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
deleted file mode 100644
index 1e676c1719..0000000000
--- a/core/src/test/scala/org/apache/spark/scheduler/local/LocalSchedulerSuite.scala
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.scheduler.local
-
-import java.util.concurrent.Semaphore
-import java.util.concurrent.CountDownLatch
-
-import scala.collection.mutable.HashMap
-
-import org.scalatest.{BeforeAndAfterEach, FunSuite}
-
-import org.apache.spark._
-
-
-class Lock() {
- var finished = false
- def jobWait() = {
- synchronized {
- while(!finished) {
- this.wait()
- }
- }
- }
-
- def jobFinished() = {
- synchronized {
- finished = true
- this.notifyAll()
- }
- }
-}
-
-object TaskThreadInfo {
- val threadToLock = HashMap[Int, Lock]()
- val threadToRunning = HashMap[Int, Boolean]()
- val threadToStarted = HashMap[Int, CountDownLatch]()
-}
-
-/*
- * 1. each thread contains one job.
- * 2. each job contains one stage.
- * 3. each stage only contains one task.
- * 4. each task(launched) must be lanched orderly(using threadToStarted) to make sure
- * it will get cpu core resource, and will wait to finished after user manually
- * release "Lock" and then cluster will contain another free cpu cores.
- * 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 with BeforeAndAfterEach {
-
- override def afterEach() {
- super.afterEach()
- System.clearProperty("spark.scheduler.mode")
- }
-
- def createThread(threadIndex: Int, poolName: String, sc: SparkContext, sem: Semaphore) {
-
- TaskThreadInfo.threadToRunning(threadIndex) = false
- val nums = sc.parallelize(threadIndex to threadIndex, 1)
- TaskThreadInfo.threadToLock(threadIndex) = new Lock()
- TaskThreadInfo.threadToStarted(threadIndex) = new CountDownLatch(1)
- new Thread {
- if (poolName != null) {
- sc.setLocalProperty("spark.scheduler.pool", poolName)
- }
- override def run() {
- val ans = nums.map(number => {
- TaskThreadInfo.threadToRunning(number) = true
- TaskThreadInfo.threadToStarted(number).countDown()
- TaskThreadInfo.threadToLock(number).jobWait()
- TaskThreadInfo.threadToRunning(number) = false
- number
- }).collect()
- assert(ans.toList === List(threadIndex))
- sem.release()
- }
- }.start()
- }
-
- test("Local FIFO scheduler end-to-end test") {
- System.setProperty("spark.scheduler.mode", "FIFO")
- sc = new SparkContext("local[4]", "test")
- val sem = new Semaphore(0)
-
- createThread(1,null,sc,sem)
- TaskThreadInfo.threadToStarted(1).await()
- createThread(2,null,sc,sem)
- TaskThreadInfo.threadToStarted(2).await()
- createThread(3,null,sc,sem)
- TaskThreadInfo.threadToStarted(3).await()
- createThread(4,null,sc,sem)
- TaskThreadInfo.threadToStarted(4).await()
- // thread 5 and 6 (stage pending)must meet following two points
- // 1. stages (taskSetManager) of jobs in thread 5 and 6 should be add to taskSetManager
- // queue before executing TaskThreadInfo.threadToLock(1).jobFinished()
- // 2. priority of stage in thread 5 should be prior to priority of stage in thread 6
- // So I just use "sleep" 1s here for each thread.
- // TODO: any better solution?
- createThread(5,null,sc,sem)
- Thread.sleep(1000)
- createThread(6,null,sc,sem)
- Thread.sleep(1000)
-
- assert(TaskThreadInfo.threadToRunning(1) === true)
- assert(TaskThreadInfo.threadToRunning(2) === true)
- assert(TaskThreadInfo.threadToRunning(3) === true)
- assert(TaskThreadInfo.threadToRunning(4) === true)
- assert(TaskThreadInfo.threadToRunning(5) === false)
- assert(TaskThreadInfo.threadToRunning(6) === false)
-
- TaskThreadInfo.threadToLock(1).jobFinished()
- TaskThreadInfo.threadToStarted(5).await()
-
- assert(TaskThreadInfo.threadToRunning(1) === false)
- assert(TaskThreadInfo.threadToRunning(2) === true)
- assert(TaskThreadInfo.threadToRunning(3) === true)
- assert(TaskThreadInfo.threadToRunning(4) === true)
- assert(TaskThreadInfo.threadToRunning(5) === true)
- assert(TaskThreadInfo.threadToRunning(6) === false)
-
- TaskThreadInfo.threadToLock(3).jobFinished()
- TaskThreadInfo.threadToStarted(6).await()
-
- assert(TaskThreadInfo.threadToRunning(1) === false)
- assert(TaskThreadInfo.threadToRunning(2) === true)
- assert(TaskThreadInfo.threadToRunning(3) === false)
- assert(TaskThreadInfo.threadToRunning(4) === true)
- assert(TaskThreadInfo.threadToRunning(5) === true)
- assert(TaskThreadInfo.threadToRunning(6) === true)
-
- TaskThreadInfo.threadToLock(2).jobFinished()
- TaskThreadInfo.threadToLock(4).jobFinished()
- TaskThreadInfo.threadToLock(5).jobFinished()
- TaskThreadInfo.threadToLock(6).jobFinished()
- sem.acquire(6)
- }
-
- 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)
-
- createThread(10,"1",sc,sem)
- TaskThreadInfo.threadToStarted(10).await()
- createThread(20,"2",sc,sem)
- TaskThreadInfo.threadToStarted(20).await()
- createThread(30,"3",sc,sem)
- TaskThreadInfo.threadToStarted(30).await()
-
- assert(TaskThreadInfo.threadToRunning(10) === true)
- assert(TaskThreadInfo.threadToRunning(20) === true)
- assert(TaskThreadInfo.threadToRunning(30) === true)
-
- createThread(11,"1",sc,sem)
- TaskThreadInfo.threadToStarted(11).await()
- createThread(21,"2",sc,sem)
- TaskThreadInfo.threadToStarted(21).await()
- createThread(31,"3",sc,sem)
- TaskThreadInfo.threadToStarted(31).await()
-
- assert(TaskThreadInfo.threadToRunning(11) === true)
- assert(TaskThreadInfo.threadToRunning(21) === true)
- assert(TaskThreadInfo.threadToRunning(31) === true)
-
- createThread(12,"1",sc,sem)
- TaskThreadInfo.threadToStarted(12).await()
- createThread(22,"2",sc,sem)
- TaskThreadInfo.threadToStarted(22).await()
- createThread(32,"3",sc,sem)
-
- assert(TaskThreadInfo.threadToRunning(12) === true)
- assert(TaskThreadInfo.threadToRunning(22) === true)
- assert(TaskThreadInfo.threadToRunning(32) === false)
-
- TaskThreadInfo.threadToLock(10).jobFinished()
- TaskThreadInfo.threadToStarted(32).await()
-
- assert(TaskThreadInfo.threadToRunning(32) === true)
-
- //1. Similar with above scenario, sleep 1s for stage of 23 and 33 to be added to taskSetManager
- // queue so that cluster will assign free cpu core to stage 23 after stage 11 finished.
- //2. priority of 23 and 33 will be meaningless as using fair scheduler here.
- createThread(23,"2",sc,sem)
- createThread(33,"3",sc,sem)
- Thread.sleep(1000)
-
- TaskThreadInfo.threadToLock(11).jobFinished()
- TaskThreadInfo.threadToStarted(23).await()
-
- assert(TaskThreadInfo.threadToRunning(23) === true)
- assert(TaskThreadInfo.threadToRunning(33) === false)
-
- TaskThreadInfo.threadToLock(12).jobFinished()
- TaskThreadInfo.threadToStarted(33).await()
-
- assert(TaskThreadInfo.threadToRunning(33) === true)
-
- TaskThreadInfo.threadToLock(20).jobFinished()
- TaskThreadInfo.threadToLock(21).jobFinished()
- TaskThreadInfo.threadToLock(22).jobFinished()
- TaskThreadInfo.threadToLock(23).jobFinished()
- TaskThreadInfo.threadToLock(30).jobFinished()
- TaskThreadInfo.threadToLock(31).jobFinished()
- TaskThreadInfo.threadToLock(32).jobFinished()
- TaskThreadInfo.threadToLock(33).jobFinished()
-
- sem.acquire(11)
- }
-}
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 c016c51171..3898583275 100644
--- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala
@@ -22,12 +22,15 @@ import scala.collection.mutable
import com.esotericsoftware.kryo.Kryo
import org.scalatest.FunSuite
-import org.apache.spark.SharedSparkContext
+import org.apache.spark.{SparkConf, SharedSparkContext}
import org.apache.spark.serializer.KryoTest._
class KryoSerializerSuite extends FunSuite with SharedSparkContext {
+ conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+ conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName)
+
test("basic types") {
- val ser = (new KryoSerializer).newInstance()
+ val ser = new KryoSerializer(conf).newInstance()
def check[T](t: T) {
assert(ser.deserialize[T](ser.serialize(t)) === t)
}
@@ -57,7 +60,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
}
test("pairs") {
- val ser = (new KryoSerializer).newInstance()
+ val ser = new KryoSerializer(conf).newInstance()
def check[T](t: T) {
assert(ser.deserialize[T](ser.serialize(t)) === t)
}
@@ -81,7 +84,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
}
test("Scala data structures") {
- val ser = (new KryoSerializer).newInstance()
+ val ser = new KryoSerializer(conf).newInstance()
def check[T](t: T) {
assert(ser.deserialize[T](ser.serialize(t)) === t)
}
@@ -104,7 +107,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
}
test("ranges") {
- val ser = (new KryoSerializer).newInstance()
+ val ser = new KryoSerializer(conf).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
@@ -125,9 +128,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
}
test("custom registrator") {
- System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
-
- val ser = (new KryoSerializer).newInstance()
+ val ser = new KryoSerializer(conf).newInstance()
def check[T](t: T) {
assert(ser.deserialize[T](ser.serialize(t)) === t)
}
@@ -172,6 +173,10 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
assert (sc.parallelize( Array((1, 11), (2, 22), (3, 33)) ).collect().head === (1, 11))
}
+ test("kryo with SerializableHyperLogLog") {
+ assert(sc.parallelize( Array(1, 2, 3, 2, 3, 3, 2, 3, 1) ).countApproxDistinct(0.01) === 3)
+ }
+
test("kryo with reduce") {
val control = 1 :: 2 :: Nil
val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_))
@@ -186,18 +191,6 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext {
.fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x
assert(10 + control.sum === result)
}
-
- override def beforeAll() {
- System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
- System.setProperty("spark.kryo.registrator", classOf[MyRegistrator].getName)
- super.beforeAll()
- }
-
- override def afterAll() {
- super.afterAll()
- System.clearProperty("spark.kryo.registrator")
- System.clearProperty("spark.serializer")
- }
}
object KryoTest {
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
index cb76275e39..b647e8a672 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
@@ -39,7 +39,7 @@ class BlockIdSuite extends FunSuite {
fail()
} catch {
case e: IllegalStateException => // OK
- case _ => fail()
+ case _: Throwable => fail()
}
}
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index 484a654108..f60ce270c7 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -31,41 +31,42 @@ import org.scalatest.time.SpanSugar._
import org.apache.spark.util.{SizeEstimator, Utils, AkkaUtils, ByteBufferInputStream}
import org.apache.spark.serializer.{JavaSerializer, KryoSerializer}
+import org.apache.spark.{SparkConf, SparkContext}
class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester {
+ private val conf = new SparkConf(false)
var store: BlockManager = null
var store2: BlockManager = null
var actorSystem: ActorSystem = null
var master: BlockManagerMaster = null
var oldArch: String = null
- var oldOops: String = null
- var oldHeartBeat: String = null
// Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
- System.setProperty("spark.kryoserializer.buffer.mb", "1")
- val serializer = new KryoSerializer
+ conf.set("spark.kryoserializer.buffer.mb", "1")
+ val serializer = new KryoSerializer(conf)
// 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)
+ val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf)
this.actorSystem = actorSystem
- System.setProperty("spark.driver.port", boundPort.toString)
- System.setProperty("spark.hostPort", "localhost:" + boundPort)
+ conf.set("spark.driver.port", boundPort.toString)
+ conf.set("spark.hostPort", "localhost:" + boundPort)
master = new BlockManagerMaster(
- actorSystem.actorOf(Props(new BlockManagerMasterActor(true))))
+ actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf)
// Set the arch to 64-bit and compressedOops to true to get a deterministic test-case
oldArch = System.setProperty("os.arch", "amd64")
- oldOops = System.setProperty("spark.test.useCompressedOops", "true")
- oldHeartBeat = System.setProperty("spark.storage.disableBlockManagerHeartBeat", "true")
+ conf.set("os.arch", "amd64")
+ conf.set("spark.test.useCompressedOops", "true")
+ conf.set("spark.storage.disableBlockManagerHeartBeat", "true")
val initialize = PrivateMethod[Unit]('initialize)
SizeEstimator invokePrivate initialize()
// Set some value ...
- System.setProperty("spark.hostPort", Utils.localHostName() + ":" + 1111)
+ conf.set("spark.hostPort", Utils.localHostName() + ":" + 1111)
}
after {
@@ -86,16 +87,12 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
master = null
if (oldArch != null) {
- System.setProperty("os.arch", oldArch)
+ conf.set("os.arch", oldArch)
} else {
System.clearProperty("os.arch")
}
- if (oldOops != null) {
- System.setProperty("spark.test.useCompressedOops", oldOops)
- } else {
- System.clearProperty("spark.test.useCompressedOops")
- }
+ System.clearProperty("spark.test.useCompressedOops")
}
test("StorageLevel object caching") {
@@ -133,7 +130,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("master + 1 manager interaction") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -163,8 +160,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("master + 2 managers interaction") {
- store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
- store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer, 2000)
+ store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf)
+ store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf)
val peers = master.getPeers(store.blockManagerId, 1)
assert(peers.size === 1, "master did not return the other manager as a peer")
@@ -179,7 +176,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("removing block") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -227,7 +224,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("removing rdd") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -261,7 +258,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
test("reregistration on heart beat") {
val heartBeat = PrivateMethod[Unit]('heartBeat)
- store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
val a1 = new Array[Byte](400)
store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
@@ -277,7 +274,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("reregistration on block update") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
@@ -296,7 +293,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
test("reregistration doesn't dead lock") {
val heartBeat = PrivateMethod[Unit]('heartBeat)
- store = new BlockManager("<driver>", actorSystem, master, serializer, 2000)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 2000, conf)
val a1 = new Array[Byte](400)
val a2 = List(new Array[Byte](400))
@@ -333,7 +330,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("in-memory LRU storage") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -352,7 +349,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("in-memory LRU storage with serialization") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -371,7 +368,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("in-memory LRU for partitions of same RDD") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -390,7 +387,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("in-memory LRU for partitions of multiple RDDs") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
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)
@@ -413,7 +410,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("on-disk storage") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -426,7 +423,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("disk and memory storage") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -441,7 +438,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("disk and memory storage with getLocalBytes") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -456,7 +453,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("disk and memory storage with serialization") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -471,7 +468,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("disk and memory storage with serialization and getLocalBytes") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -486,7 +483,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("LRU with mixed storage levels") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
@@ -511,7 +508,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("in-memory LRU with streams") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val list1 = List(new Array[Byte](200), new Array[Byte](200))
val list2 = List(new Array[Byte](200), new Array[Byte](200))
val list3 = List(new Array[Byte](200), new Array[Byte](200))
@@ -535,7 +532,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("LRU with mixed storage levels and streams") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 1200, conf)
val list1 = List(new Array[Byte](200), new Array[Byte](200))
val list2 = List(new Array[Byte](200), new Array[Byte](200))
val list3 = List(new Array[Byte](200), new Array[Byte](200))
@@ -581,7 +578,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("overly large block") {
- store = new BlockManager("<driver>", actorSystem, master, serializer, 500)
+ store = new BlockManager("<driver>", actorSystem, master, serializer, 500, conf)
store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY)
assert(store.getSingle("a1") === None, "a1 was in store")
store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK)
@@ -591,53 +588,53 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
test("block compression") {
try {
- System.setProperty("spark.shuffle.compress", "true")
- store = new BlockManager("exec1", actorSystem, master, serializer, 2000)
+ conf.set("spark.shuffle.compress", "true")
+ store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf)
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)
+ conf.set("spark.shuffle.compress", "false")
+ store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf)
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)
+ conf.set("spark.broadcast.compress", "true")
+ store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf)
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)
+ conf.set("spark.broadcast.compress", "false")
+ store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf)
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)
+ conf.set("spark.rdd.compress", "true")
+ store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf)
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)
+ conf.set("spark.rdd.compress", "false")
+ store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf)
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
// Check that any other block types are also kept uncompressed
- store = new BlockManager("exec7", actorSystem, master, serializer, 2000)
+ store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf)
store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY)
assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed")
store.stop()
@@ -651,7 +648,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
test("block store put failure") {
// Use Java serializer so we can create an unserializable error.
- store = new BlockManager("<driver>", actorSystem, master, new JavaSerializer, 1200)
+ store = new BlockManager("<driver>", actorSystem, master, new JavaSerializer(conf), 1200, conf)
// The put should fail since a1 is not serializable.
class UnserializableClass
diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
index 0b9056344c..829f389460 100644
--- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
@@ -1,14 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.storage
-import java.io.{FileWriter, File}
+import java.io.{File, FileWriter}
import scala.collection.mutable
import com.google.common.io.Files
+import org.apache.spark.SparkConf
import org.scalatest.{BeforeAndAfterEach, FunSuite}
class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach {
-
+ private val testConf = new SparkConf(false)
val rootDir0 = Files.createTempDir()
rootDir0.deleteOnExit()
val rootDir1 = Files.createTempDir()
@@ -16,7 +34,12 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach {
val rootDirs = rootDir0.getName + "," + rootDir1.getName
println("Created root dirs: " + rootDirs)
+ // This suite focuses primarily on consolidation features,
+ // so we coerce consolidation if not already enabled.
+ testConf.set("spark.shuffle.consolidateFiles", "true")
+
val shuffleBlockManager = new ShuffleBlockManager(null) {
+ override def conf = testConf.clone
var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]()
override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id)
}
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 8f0ec6683b..3764f4d1a0 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala
@@ -34,7 +34,6 @@ class UISuite extends FunSuite {
}
val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("localhost", startPort, Seq())
val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("localhost", startPort, Seq())
-
// Allow some wiggle room in case ports on the machine are under contention
assert(boundPort1 > startPort && boundPort1 < startPort + 10)
assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10)
diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala
new file mode 100644
index 0000000000..67a57a0e7f
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.ui.jobs
+
+import org.scalatest.FunSuite
+import org.apache.spark.scheduler._
+import org.apache.spark.{LocalSparkContext, SparkContext, Success}
+import org.apache.spark.scheduler.SparkListenerTaskStart
+import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics}
+
+class JobProgressListenerSuite extends FunSuite with LocalSparkContext {
+ test("test executor id to summary") {
+ val sc = new SparkContext("local", "test")
+ val listener = new JobProgressListener(sc)
+ val taskMetrics = new TaskMetrics()
+ val shuffleReadMetrics = new ShuffleReadMetrics()
+
+ // nothing in it
+ assert(listener.stageIdToExecutorSummaries.size == 0)
+
+ // finish this task, should get updated shuffleRead
+ shuffleReadMetrics.remoteBytesRead = 1000
+ taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics)
+ var taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL)
+ taskInfo.finishTime = 1
+ listener.onTaskEnd(new SparkListenerTaskEnd(
+ new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics))
+ assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail())
+ .shuffleRead == 1000)
+
+ // finish a task with unknown executor-id, nothing should happen
+ taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL)
+ taskInfo.finishTime = 1
+ listener.onTaskEnd(new SparkListenerTaskEnd(
+ new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics))
+ assert(listener.stageIdToExecutorSummaries.size == 1)
+
+ // finish this task, should get updated duration
+ shuffleReadMetrics.remoteBytesRead = 1000
+ taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics)
+ taskInfo = new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL)
+ taskInfo.finishTime = 1
+ listener.onTaskEnd(new SparkListenerTaskEnd(
+ new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics))
+ assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail())
+ .shuffleRead == 2000)
+
+ // finish this task, should get updated duration
+ shuffleReadMetrics.remoteBytesRead = 1000
+ taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics)
+ taskInfo = new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL)
+ taskInfo.finishTime = 1
+ listener.onTaskEnd(new SparkListenerTaskEnd(
+ new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics))
+ assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail())
+ .shuffleRead == 1000)
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
index 4e40dcbdee..11ebdc352b 100644
--- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.util
import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfterAll
import org.scalatest.PrivateMethodTester
+import org.apache.spark.SparkContext
class DummyClass1 {}
@@ -63,54 +64,53 @@ class SizeEstimatorSuite
}
test("simple classes") {
- assert(SizeEstimator.estimate(new DummyClass1) === 16)
- assert(SizeEstimator.estimate(new DummyClass2) === 16)
- assert(SizeEstimator.estimate(new DummyClass3) === 24)
- assert(SizeEstimator.estimate(new DummyClass4(null)) === 24)
- assert(SizeEstimator.estimate(new DummyClass4(new DummyClass3)) === 48)
+ expectResult(16)(SizeEstimator.estimate(new DummyClass1))
+ expectResult(16)(SizeEstimator.estimate(new DummyClass2))
+ expectResult(24)(SizeEstimator.estimate(new DummyClass3))
+ expectResult(24)(SizeEstimator.estimate(new DummyClass4(null)))
+ expectResult(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
}
// NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors
// (Sun vs IBM). Use a DummyString class to make tests deterministic.
test("strings") {
- assert(SizeEstimator.estimate(DummyString("")) === 40)
- assert(SizeEstimator.estimate(DummyString("a")) === 48)
- assert(SizeEstimator.estimate(DummyString("ab")) === 48)
- assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56)
+ expectResult(40)(SizeEstimator.estimate(DummyString("")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("a")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("ab")))
+ expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
}
test("primitive arrays") {
- assert(SizeEstimator.estimate(new Array[Byte](10)) === 32)
- assert(SizeEstimator.estimate(new Array[Char](10)) === 40)
- assert(SizeEstimator.estimate(new Array[Short](10)) === 40)
- assert(SizeEstimator.estimate(new Array[Int](10)) === 56)
- assert(SizeEstimator.estimate(new Array[Long](10)) === 96)
- assert(SizeEstimator.estimate(new Array[Float](10)) === 56)
- assert(SizeEstimator.estimate(new Array[Double](10)) === 96)
- assert(SizeEstimator.estimate(new Array[Int](1000)) === 4016)
- assert(SizeEstimator.estimate(new Array[Long](1000)) === 8016)
+ expectResult(32)(SizeEstimator.estimate(new Array[Byte](10)))
+ expectResult(40)(SizeEstimator.estimate(new Array[Char](10)))
+ expectResult(40)(SizeEstimator.estimate(new Array[Short](10)))
+ expectResult(56)(SizeEstimator.estimate(new Array[Int](10)))
+ expectResult(96)(SizeEstimator.estimate(new Array[Long](10)))
+ expectResult(56)(SizeEstimator.estimate(new Array[Float](10)))
+ expectResult(96)(SizeEstimator.estimate(new Array[Double](10)))
+ expectResult(4016)(SizeEstimator.estimate(new Array[Int](1000)))
+ expectResult(8016)(SizeEstimator.estimate(new Array[Long](1000)))
}
test("object arrays") {
// Arrays containing nulls should just have one pointer per element
- assert(SizeEstimator.estimate(new Array[String](10)) === 56)
- assert(SizeEstimator.estimate(new Array[AnyRef](10)) === 56)
-
+ expectResult(56)(SizeEstimator.estimate(new Array[String](10)))
+ expectResult(56)(SizeEstimator.estimate(new Array[AnyRef](10)))
// For object arrays with non-null elements, each object should take one pointer plus
// however many bytes that class takes. (Note that Array.fill calls the code in its
// second parameter separately for each object, so we get distinct objects.)
- assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)) === 216)
- assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)) === 216)
- assert(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)) === 296)
- assert(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)) === 56)
+ expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass1)))
+ expectResult(216)(SizeEstimator.estimate(Array.fill(10)(new DummyClass2)))
+ expectResult(296)(SizeEstimator.estimate(Array.fill(10)(new DummyClass3)))
+ expectResult(56)(SizeEstimator.estimate(Array(new DummyClass1, new DummyClass2)))
// Past size 100, our samples 100 elements, but we should still get the right size.
- assert(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)) === 28016)
+ expectResult(28016)(SizeEstimator.estimate(Array.fill(1000)(new DummyClass3)))
// If an array contains the *same* element many times, we should only count it once.
val d1 = new DummyClass1
- assert(SizeEstimator.estimate(Array.fill(10)(d1)) === 72) // 10 pointers plus 8-byte object
- assert(SizeEstimator.estimate(Array.fill(100)(d1)) === 432) // 100 pointers plus 8-byte object
+ expectResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object
+ expectResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object
// Same thing with huge array containing the same element many times. Note that this won't
// return exactly 4032 because it can't tell that *all* the elements will equal the first
@@ -128,11 +128,10 @@ class SizeEstimatorSuite
val initialize = PrivateMethod[Unit]('initialize)
SizeEstimator invokePrivate initialize()
- assert(SizeEstimator.estimate(DummyString("")) === 40)
- assert(SizeEstimator.estimate(DummyString("a")) === 48)
- assert(SizeEstimator.estimate(DummyString("ab")) === 48)
- assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 56)
-
+ expectResult(40)(SizeEstimator.estimate(DummyString("")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("a")))
+ expectResult(48)(SizeEstimator.estimate(DummyString("ab")))
+ expectResult(56)(SizeEstimator.estimate(DummyString("abcdefgh")))
resetOrClear("os.arch", arch)
}
@@ -141,14 +140,13 @@ class SizeEstimatorSuite
test("64-bit arch with no compressed oops") {
val arch = System.setProperty("os.arch", "amd64")
val oops = System.setProperty("spark.test.useCompressedOops", "false")
-
val initialize = PrivateMethod[Unit]('initialize)
SizeEstimator invokePrivate initialize()
- assert(SizeEstimator.estimate(DummyString("")) === 56)
- assert(SizeEstimator.estimate(DummyString("a")) === 64)
- assert(SizeEstimator.estimate(DummyString("ab")) === 64)
- assert(SizeEstimator.estimate(DummyString("abcdefgh")) === 72)
+ expectResult(56)(SizeEstimator.estimate(DummyString("")))
+ expectResult(64)(SizeEstimator.estimate(DummyString("a")))
+ expectResult(64)(SizeEstimator.estimate(DummyString("ab")))
+ expectResult(72)(SizeEstimator.estimate(DummyString("abcdefgh")))
resetOrClear("os.arch", arch)
resetOrClear("spark.test.useCompressedOops", oops)
diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
index ca3f684668..e9b62ea70d 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala
@@ -1,9 +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.
+ */
+
package org.apache.spark.util.collection
import scala.collection.mutable.HashSet
import org.scalatest.FunSuite
-
-class OpenHashMapSuite extends FunSuite {
+import org.scalatest.matchers.ShouldMatchers
+import org.apache.spark.util.SizeEstimator
+
+class OpenHashMapSuite extends FunSuite with ShouldMatchers {
+
+ test("size for specialized, primitive value (int)") {
+ val capacity = 1024
+ val map = new OpenHashMap[String, Int](capacity)
+ val actualSize = SizeEstimator.estimate(map)
+ // 64 bit for pointers, 32 bit for ints, and 1 bit for the bitset.
+ val expectedSize = capacity * (64 + 32 + 1) / 8
+ // Make sure we are not allocating a significant amount of memory beyond our expected.
+ actualSize should be <= (expectedSize * 1.1).toLong
+ }
test("initialization") {
val goodMap1 = new OpenHashMap[String, Int](1)
diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
index 4e11e8a628..1b24f8f287 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala
@@ -1,9 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.util.collection
import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+
+import org.apache.spark.util.SizeEstimator
+
+class OpenHashSetSuite extends FunSuite with ShouldMatchers {
-class OpenHashSetSuite extends FunSuite {
+ test("size for specialized, primitive int") {
+ val loadFactor = 0.7
+ val set = new OpenHashSet[Int](64, loadFactor)
+ for (i <- 0 until 1024) {
+ set.add(i)
+ }
+ assert(set.size === 1024)
+ assert(set.capacity > 1024)
+ val actualSize = SizeEstimator.estimate(set)
+ // 32 bits for the ints + 1 bit for the bitset
+ val expectedSize = set.capacity * (32 + 1) / 8
+ // Make sure we are not allocating a significant amount of memory beyond our expected.
+ actualSize should be <= (expectedSize * 1.1).toLong
+ }
test("primitive int") {
val set = new OpenHashSet[Int]
diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala
index dfd6aed2c4..3b60decee9 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashSetSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveKeyOpenHashMapSuite.scala
@@ -1,9 +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.
+ */
+
package org.apache.spark.util.collection
import scala.collection.mutable.HashSet
import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+import org.apache.spark.util.SizeEstimator
+
+class PrimitiveKeyOpenHashMapSuite extends FunSuite with ShouldMatchers {
-class PrimitiveKeyOpenHashSetSuite extends FunSuite {
+ test("size for specialized, primitive key, value (int, int)") {
+ val capacity = 1024
+ val map = new PrimitiveKeyOpenHashMap[Int, Int](capacity)
+ val actualSize = SizeEstimator.estimate(map)
+ // 32 bit for keys, 32 bit for values, and 1 bit for the bitset.
+ val expectedSize = capacity * (32 + 32 + 1) / 8
+ // Make sure we are not allocating a significant amount of memory beyond our expected.
+ actualSize should be <= (expectedSize * 1.1).toLong
+ }
test("initialization") {
val goodMap1 = new PrimitiveKeyOpenHashMap[Int, Int](1)
diff --git a/kmeans_data.txt b/data/kmeans_data.txt
index 338664f78d..338664f78d 100644
--- a/kmeans_data.txt
+++ b/data/kmeans_data.txt
diff --git a/lr_data.txt b/data/lr_data.txt
index d4df0634e0..d4df0634e0 100644
--- a/lr_data.txt
+++ b/data/lr_data.txt
diff --git a/pagerank_data.txt b/data/pagerank_data.txt
index 95755ab8f5..95755ab8f5 100644
--- a/pagerank_data.txt
+++ b/data/pagerank_data.txt
diff --git a/docs/_config.yml b/docs/_config.yml
index 48ecb8d0c9..11d18f0ac2 100644
--- a/docs/_config.yml
+++ b/docs/_config.yml
@@ -4,7 +4,7 @@ markdown: kramdown
# These allow the documentation to be updated with nerw releases
# of Spark, Scala, and Mesos.
SPARK_VERSION: 0.9.0-incubating-SNAPSHOT
-SPARK_VERSION_SHORT: 0.9.0-SNAPSHOT
-SCALA_VERSION: 2.9.3
+SPARK_VERSION_SHORT: 0.9.0
+SCALA_VERSION: 2.10
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 0c1d657cde..ad7969d012 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -74,12 +74,12 @@
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">API Docs<b class="caret"></b></a>
<ul class="dropdown-menu">
- <li><a href="api/core/index.html">Spark Core for Java/Scala</a></li>
+ <li><a href="api/core/index.html#org.apache.spark.package">Spark Core for Java/Scala</a></li>
<li><a href="api/pyspark/index.html">Spark Core for Python</a></li>
<li class="divider"></li>
- <li><a href="api/streaming/index.html">Spark Streaming</a></li>
- <li><a href="api/mllib/index.html">MLlib (Machine Learning)</a></li>
- <li><a href="api/bagel/index.html">Bagel (Pregel on Spark)</a></li>
+ <li><a href="api/streaming/index.html#org.apache.spark.streaming.package">Spark Streaming</a></li>
+ <li><a href="api/mllib/index.html#org.apache.spark.mllib.package">MLlib (Machine Learning)</a></li>
+ <li><a href="api/bagel/index.html#org.apache.spark.bagel.package">Bagel (Pregel on Spark)</a></li>
</ul>
</li>
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index c574ea7f5c..431de909cb 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -35,7 +35,7 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1')
# Copy over the scaladoc from each project into the docs directory.
# This directory will be copied over to _site when `jekyll` command is run.
projects.each do |project_name|
- source = "../" + project_name + "/target/scala-2.9.3/api"
+ source = "../" + project_name + "/target/scala-2.10/api"
dest = "api/" + project_name
puts "echo making directory " + dest
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
index 140190a38c..c4f1f6d6ad 100644
--- a/docs/bagel-programming-guide.md
+++ b/docs/bagel-programming-guide.md
@@ -106,7 +106,7 @@ _Example_
## Operations
-Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/spark/bagel/Bagel.scala) for details.
+Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala) for details.
### Actions
@@ -157,8 +157,8 @@ trait Message[K] {
# Where to Go from Here
-Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `run-example` script included in Spark; e.g.:
+Two example jobs, PageRank and shortest path, are included in `examples/src/main/scala/org/apache/spark/examples/bagel`. You can run them by passing the class name to the `bin/run-example` script included in Spark; e.g.:
- ./run-example org.apache.spark.examples.bagel.WikipediaPageRank
+ ./bin/run-example org.apache.spark.examples.bagel.WikipediaPageRank
Each example program prints usage help when run without any arguments.
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index 19c01e179f..b9ff0af76f 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -37,14 +37,16 @@ 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 enable the "hadoop2-yarn" profile and set the "yarn.version" property:
+For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you should enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property:
# Apache Hadoop 2.0.5-alpha
- $ mvn -Phadoop2-yarn -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package
+ $ mvn -Pyarn-alpha -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 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
+ $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package
+ # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer
+ $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package
## Spark Tests in Maven ##
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index 5927f736f3..e16703292c 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -45,7 +45,7 @@ The system currently supports three cluster managers:
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.
+* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone
cluster on Amazon EC2.
diff --git a/docs/configuration.md b/docs/configuration.md
index 97183bafdb..1d6c3d1633 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -3,26 +3,37 @@ layout: global
title: Spark Configuration
---
-Spark provides three main locations to configure the system:
+Spark provides three locations to configure the system:
-* [Java system properties](#system-properties), which control internal configuration parameters and can be set
- either programmatically (by calling `System.setProperty` *before* creating a `SparkContext`) or through
- JVM arguments.
-* [Environment variables](#environment-variables) for configuring per-machine settings such as the IP address,
- which can be set in the `conf/spark-env.sh` script.
-* [Logging configuration](#configuring-logging), which is done through `log4j.properties`.
+* [Spark properties](#spark-properties) control most application parameters and can be set by passing
+ a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java
+ system properties.
+* [Environment variables](#environment-variables) can be used to set per-machine settings, such as
+ the IP address, through the `conf/spark-env.sh` script on each node.
+* [Logging](#configuring-logging) can be configured through `log4j.properties`.
-# System Properties
+# Spark Properties
-To set a system property for configuring Spark, you need to either pass it with a -D flag to the JVM (for example `java -Dspark.cores.max=5 MyProgram`) or call `System.setProperty` in your code *before* creating your Spark context, as follows:
+Spark properties control most application settings and are configured separately for each application.
+The preferred way to set them is by passing a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+class to your SparkContext constructor.
+Alternatively, Spark will also load them from Java system properties (for compatibility with old versions
+of Spark) and from a [`spark.conf` file](#configuration-files) on your classpath.
+
+SparkConf lets you configure most of the common properties to initialize a cluster (e.g., master URL and
+application name), as well as arbitrary key-value pairs through the `set()` method. For example, we could
+initialize an application as follows:
{% highlight scala %}
-System.setProperty("spark.cores.max", "5")
-val sc = new SparkContext(...)
+val conf = new SparkConf()
+ .setMaster("local")
+ .setAppName("My application")
+ .set("spark.executor.memory", "1g")
+val sc = new SparkContext(conf)
{% endhighlight %}
-Most of the configurable system properties control internal settings that have reasonable default values. However,
+Most of the properties control internal settings that have reasonable default values. However,
there are at least five properties that you will commonly want to control:
<table class="table">
@@ -66,12 +77,14 @@ there are at least five properties that you will commonly want to control:
</tr>
<tr>
<td>spark.cores.max</td>
- <td>(infinite)</td>
+ <td>(not set)</td>
<td>
When running on a <a href="spark-standalone.html">standalone deploy cluster</a> or a
<a href="running-on-mesos.html#mesos-run-modes">Mesos cluster in "coarse-grained"
- sharing mode</a>, how many CPU cores to request at most. The default will use all available cores
- offered by the cluster manager.
+ sharing mode</a>, the maximum amount of CPU cores to request for the application from
+ across the cluster (not from each machine). If not set, the default will be
+ <code>spark.deploy.defaultCores</code> on Spark's standalone cluster manager, or
+ infinite (all available cores) on Mesos.
</td>
</tr>
</table>
@@ -275,12 +288,33 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td>spark.akka.timeout</td>
- <td>20</td>
+ <td>100</td>
<td>
Communication timeout between Spark nodes, in seconds.
</td>
</tr>
<tr>
+ <td>spark.akka.heartbeat.pauses</td>
+ <td>600</td>
+ <td>
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause in seconds for akka. This can be used to control sensitivity to gc pauses. Tune this in combination of `spark.akka.heartbeat.interval` and `spark.akka.failure-detector.threshold` if you need to.
+ </td>
+</tr>
+<tr>
+ <td>spark.akka.failure-detector.threshold</td>
+ <td>300.0</td>
+ <td>
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). This maps to akka's `akka.remote.transport-failure-detector.threshold`. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.heartbeat.interval` if you need to.
+ </td>
+</tr>
+<tr>
+ <td>spark.akka.heartbeat.interval</td>
+ <td>1000</td>
+ <td>
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those.
+ </td>
+</tr>
+<tr>
<td>spark.driver.host</td>
<td>(local hostname)</td>
<td>
@@ -327,14 +361,116 @@ Apart from these, the following properties are also available, and may be useful
Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, <code>BlockManager</code> might take a performance hit.
</td>
</tr>
+<tr>
+ <td>akka.x.y....</td>
+ <td>value</td>
+ <td>
+ An arbitrary akka configuration can be set directly on spark conf and it is applied for all the ActorSystems created spark wide for that SparkContext and its assigned executors as well.
+ </td>
+</tr>
+<tr>
+ <td>spark.shuffle.consolidateFiles</td>
+ <td>false</td>
+ <td>
+ If set to "true", consolidates intermediate files created during a shuffle. Creating fewer files can improve filesystem performance for shuffles with large numbers of reduce tasks. It is recommended to set this to "true" when using ext4 or xfs filesystems. On ext3, this option might degrade performance on machines with many (>8) cores due to filesystem limitations.
+ </td>
+</tr>
+<tr>
+ <td>spark.speculation</td>
+ <td>false</td>
+ <td>
+ If set to "true", performs speculative execution of tasks. This means if one or more tasks are running slowly in a stage, they will be re-launched.
+ </td>
+</tr>
+<tr>
+ <td>spark.speculation.interval</td>
+ <td>100</td>
+ <td>
+ How often Spark will check for tasks to speculate, in milliseconds.
+ </td>
+</tr>
+<tr>
+ <td>spark.speculation.quantile</td>
+ <td>0.75</td>
+ <td>
+ Percentage of tasks which must be complete before speculation is enabled for a particular stage.
+ </td>
+</tr>
+<tr>
+ <td>spark.speculation.multiplier</td>
+ <td>1.5</td>
+ <td>
+ How many times slower a task is than the median to be considered for speculation.
+ </td>
+</tr>
+<tr>
+ <td>spark.logConf</td>
+ <td>false</td>
+ <td>
+ Log the supplied SparkConf as INFO at start of spark context.
+ </td>
+</tr>
+<tr>
+ <td>spark.deploy.spreadOut</td>
+ <td>true</td>
+ <td>
+ Whether the standalone cluster manager should spread applications out across nodes or try
+ to consolidate them onto as few nodes as possible. Spreading out is usually better for
+ data locality in HDFS, but consolidating is more efficient for compute-intensive workloads. <br/>
+ <b>Note:</b> this setting needs to be configured in the standalone cluster master, not in individual
+ applications; you can set it through <code>SPARK_JAVA_OPTS</code> in <code>spark-env.sh</code>.
+ </td>
+</tr>
+<tr>
+ <td>spark.deploy.defaultCores</td>
+ <td>(infinite)</td>
+ <td>
+ Default number of cores to give to applications in Spark's standalone mode if they don't
+ set <code>spark.cores.max</code>. If not set, applications always get all available
+ cores unless they configure <code>spark.cores.max</code> themselves.
+ Set this lower on a shared cluster to prevent users from grabbing
+ the whole cluster by default. <br/>
+ <b>Note:</b> this setting needs to be configured in the standalone cluster master, not in individual
+ applications; you can set it through <code>SPARK_JAVA_OPTS</code> in <code>spark-env.sh</code>.
+ </td>
+</tr>
</table>
+## Viewing Spark Properties
+
+The application web UI at `http://<driver>:4040` lists Spark properties in the "Environment" tab.
+This is a useful place to check to make sure that your properties have been set correctly.
+
+## Configuration Files
+
+You can also configure Spark properties through a `spark.conf` file on your Java classpath.
+Because these properties are usually application-specific, we recommend putting this fine *only* on your
+application's classpath, and not in a global Spark classpath.
+
+The `spark.conf` file uses Typesafe Config's [HOCON format](https://github.com/typesafehub/config#json-superset),
+which is a superset of Java properties files and JSON. For example, the following is a simple config file:
+
+{% highlight awk %}
+# Comments are allowed
+spark.executor.memory = 512m
+spark.serializer = org.apache.spark.serializer.KryoSerializer
+{% endhighlight %}
+
+The format also allows hierarchical nesting, as follows:
+
+{% highlight awk %}
+spark.akka {
+ threads = 8
+ timeout = 200
+}
+{% endhighlight %}
+
# Environment Variables
-Certain Spark settings can also be configured through environment variables, which are read from the `conf/spark-env.sh`
+Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh`
script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such
-as library search paths. While Java system properties can also be set here, for application settings, we recommend setting
+as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting
these properties within the application instead of in `spark-env.sh` so that different applications can use different
settings.
@@ -351,7 +487,8 @@ The following variables can be set in `spark-env.sh`:
Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend
doing that when possible.
* `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system
- properties that you'd like to pass with `-D` (e.g., `-Dspark.local.dir=/disk1,/disk2`).
+ properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this
+ machine, e.g., `-Dspark.local.dir=/disk1,/disk2`.
* Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores
to use on each machine and maximum memory.
diff --git a/docs/css/bootstrap.min.css b/docs/css/bootstrap.min.css
index ccb529eaba..30ea440c00 100644
--- a/docs/css/bootstrap.min.css
+++ b/docs/css/bootstrap.min.css
@@ -6,4 +6,4 @@
* http://www.apache.org/licenses/LICENSE-2.0
*
* Designed and built with all the love in the world @twitter by @mdo and @fat.
- */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#d4e4f4);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#d4e4f4));background-image:-webkit-linear-gradient(top,#fff,#d4e4f4);background-image:-o-linear-gradient(top,#fff,#d4e4f4);background-image:linear-gradient(to bottom,#fff,#d4e4f4);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffd4e4f4',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed}
+ */article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block}audio,canvas,video{display:inline-block;*display:inline;*zoom:1}audio:not([controls]){display:none}html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%}a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}a:hover,a:active{outline:0}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sup{top:-0.5em}sub{bottom:-0.25em}img{height:auto;max-width:100%;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic}#map_canvas img{max-width:none}button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle}button,input{*overflow:visible;line-height:normal}button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0}button,input[type="button"],input[type="reset"],input[type="submit"]{cursor:pointer;-webkit-appearance:button}input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield}input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none}textarea{overflow:auto;vertical-align:top}.clearfix{*zoom:1}.clearfix:before,.clearfix:after{display:table;line-height:0;content:""}.clearfix:after{clear:both}.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333;background-color:#fff}a{color:#08c;text-decoration:none}a:hover{color:#005580;text-decoration:underline}.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.1);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.1);box-shadow:0 1px 3px rgba(0,0,0,0.1)}.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px}.row{margin-left:-20px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:20px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.span12{width:940px}.span11{width:860px}.span10{width:780px}.span9{width:700px}.span8{width:620px}.span7{width:540px}.span6{width:460px}.span5{width:380px}.span4{width:300px}.span3{width:220px}.span2{width:140px}.span1{width:60px}.offset12{margin-left:980px}.offset11{margin-left:900px}.offset10{margin-left:820px}.offset9{margin-left:740px}.offset8{margin-left:660px}.offset7{margin-left:580px}.offset6{margin-left:500px}.offset5{margin-left:420px}.offset4{margin-left:340px}.offset3{margin-left:260px}.offset2{margin-left:180px}.offset1{margin-left:100px}.row-fluid{width:100%;*zoom:1}.row-fluid:before,.row-fluid:after{display:table;line-height:0;content:""}.row-fluid:after{clear:both}.row-fluid [class*="span"]{display:block;float:left;width:100%;min-height:30px;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.row-fluid [class*="span"]:first-child{margin-left:0}.row-fluid .span12{width:100%;*width:99.94680851063829%}.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%}.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%}.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%}.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%}.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%}.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%}.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%}.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%}.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%}.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%}.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%}.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%}.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%}.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%}.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%}.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%}.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%}.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%}.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%}.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%}.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%}.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%}.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%}.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%}.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%}.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%}.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%}.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%}.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%}.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%}.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%}.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%}.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%}.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%}.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%}[class*="span"].hide,.row-fluid [class*="span"].hide{display:none}[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right}.container{margin-right:auto;margin-left:auto;*zoom:1}.container:before,.container:after{display:table;line-height:0;content:""}.container:after{clear:both}.container-fluid{padding-right:20px;padding-left:20px;*zoom:1}.container-fluid:before,.container-fluid:after{display:table;line-height:0;content:""}.container-fluid:after{clear:both}p{margin:0 0 10px}.lead{margin-bottom:20px;font-size:20px;font-weight:200;line-height:30px}small{font-size:85%}strong{font-weight:bold}em{font-style:italic}cite{font-style:normal}.muted{color:#999}h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:1;color:inherit;text-rendering:optimizelegibility}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999}h1{font-size:36px;line-height:40px}h2{font-size:30px;line-height:40px}h3{font-size:24px;line-height:40px}h4{font-size:18px;line-height:20px}h5{font-size:14px;line-height:20px}h6{font-size:12px;line-height:20px}h1 small{font-size:24px}h2 small{font-size:18px}h3 small{font-size:14px}h4 small{font-size:14px}.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eee}ul,ol{padding:0;margin:0 0 10px 25px}ul ul,ul ol,ol ol,ol ul{margin-bottom:0}li{line-height:20px}ul.unstyled,ol.unstyled{margin-left:0;list-style:none}dl{margin-bottom:20px}dt,dd{line-height:20px}dt{font-weight:bold}dd{margin-left:10px}.dl-horizontal dt{float:left;width:120px;overflow:hidden;clear:left;text-align:right;text-overflow:ellipsis;white-space:nowrap}.dl-horizontal dd{margin-left:130px}hr{margin:20px 0;border:0;border-top:1px solid #eee;border-bottom:1px solid #fff}abbr[title]{cursor:help;border-bottom:1px dotted #999}abbr.initialism{font-size:90%;text-transform:uppercase}blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eee}blockquote p{margin-bottom:0;font-size:16px;font-weight:300;line-height:25px}blockquote small{display:block;line-height:20px;color:#999}blockquote small:before{content:'\2014 \00A0'}blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eee;border-left:0}blockquote.pull-right p,blockquote.pull-right small{text-align:right}blockquote.pull-right small:before{content:''}blockquote.pull-right small:after{content:'\00A0 \2014'}q:before,q:after,blockquote:before,blockquote:after{content:""}address{display:block;margin-bottom:20px;font-style:normal;line-height:20px}code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8}pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}pre.prettyprint{margin-bottom:20px}pre code{padding:0;color:inherit;background-color:transparent;border:0}.pre-scrollable{max-height:340px;overflow-y:scroll}form{margin:0 0 20px}fieldset{padding:0;margin:0;border:0}legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333;border:0;border-bottom:1px solid #e5e5e5}legend small{font-size:15px;color:#999}label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px}input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif}label{display:block;margin-bottom:5px}select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:9px;font-size:14px;line-height:20px;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}input,textarea{width:210px}textarea{height:auto}textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#fff;border:1px solid #ccc;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-webkit-transition:border linear .2s,box-shadow linear .2s;-moz-transition:border linear .2s,box-shadow linear .2s;-o-transition:border linear .2s,box-shadow linear .2s;transition:border linear .2s,box-shadow linear .2s}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82,168,236,0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 8px rgba(82,168,236,0.6)}input[type="radio"],input[type="checkbox"]{margin:4px 0 0;margin-top:1px \9;*margin-top:0;line-height:normal;cursor:pointer}input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto}select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px}select{width:220px;background-color:#fff;border:1px solid #bbb}select[multiple],select[size]{height:auto}select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.uneditable-input,.uneditable-textarea{color:#999;cursor:not-allowed;background-color:#fcfcfc;border-color:#ccc;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.025);box-shadow:inset 0 1px 2px rgba(0,0,0,0.025)}.uneditable-input{overflow:hidden;white-space:nowrap}.uneditable-textarea{width:auto;height:auto}input:-moz-placeholder,textarea:-moz-placeholder{color:#999}input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999}input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999}.radio,.checkbox{min-height:18px;padding-left:18px}.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-18px}.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px}.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle}.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px}.input-mini{width:60px}.input-small{width:90px}.input-medium{width:150px}.input-large{width:210px}.input-xlarge{width:270px}.input-xxlarge{width:530px}input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0}.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:926px}input.span11,textarea.span11,.uneditable-input.span11{width:846px}input.span10,textarea.span10,.uneditable-input.span10{width:766px}input.span9,textarea.span9,.uneditable-input.span9{width:686px}input.span8,textarea.span8,.uneditable-input.span8{width:606px}input.span7,textarea.span7,.uneditable-input.span7{width:526px}input.span6,textarea.span6,.uneditable-input.span6{width:446px}input.span5,textarea.span5,.uneditable-input.span5{width:366px}input.span4,textarea.span4,.uneditable-input.span4{width:286px}input.span3,textarea.span3,.uneditable-input.span3{width:206px}input.span2,textarea.span2,.uneditable-input.span2{width:126px}input.span1,textarea.span1,.uneditable-input.span1{width:46px}.controls-row{*zoom:1}.controls-row:before,.controls-row:after{display:table;line-height:0;content:""}.controls-row:after{clear:both}.controls-row [class*="span"]{float:left}input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eee}input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent}.control-group.warning>label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853}.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.warning .checkbox:focus,.control-group.warning .radio:focus,.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #dbc59e}.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853}.control-group.error>label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48}.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.error .checkbox:focus,.control-group.error .radio:focus,.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #d59392}.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48}.control-group.success>label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847}.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075);box-shadow:inset 0 1px 1px rgba(0,0,0,0.075)}.control-group.success .checkbox:focus,.control-group.success .radio:focus,.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0,0,0,0.075),0 0 6px #7aba7b}.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847}input:focus:required:invalid,textarea:focus:required:invalid,select:focus:required:invalid{color:#b94a48;border-color:#ee5f5b}input:focus:required:invalid:focus,textarea:focus:required:invalid:focus,select:focus:required:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7}.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1}.form-actions:before,.form-actions:after{display:table;line-height:0;content:""}.form-actions:after{clear:both}.help-block,.help-inline{color:#595959}.help-block{display:block;margin-bottom:10px}.help-inline{display:inline-block;*display:inline;padding-left:5px;vertical-align:middle;*zoom:1}.input-append,.input-prepend{margin-bottom:5px;font-size:0;white-space:nowrap}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;font-size:14px;vertical-align:top;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2}.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #fff;background-color:#eee;border:1px solid #ccc}.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn{margin-left:-1px;vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546}.input-prepend .add-on,.input-prepend .btn{margin-right:-1px}.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-append .add-on:last-child,.input-append .btn:last-child{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0}.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px}.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;margin-bottom:0;vertical-align:middle;*zoom:1}.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none}.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block}.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0}.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle}.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0}.control-group{margin-bottom:10px}legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate}.form-horizontal .control-group{margin-bottom:20px;*zoom:1}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;line-height:0;content:""}.form-horizontal .control-group:after{clear:both}.form-horizontal .control-label{float:left;width:140px;padding-top:5px;text-align:right}.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:160px;*margin-left:0}.form-horizontal .controls:first-child{*padding-left:160px}.form-horizontal .help-block{margin-top:10px;margin-bottom:0}.form-horizontal .form-actions{padding-left:160px}table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0}.table{width:100%;margin-bottom:20px}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #ddd}.table th{font-weight:bold}.table thead th{vertical-align:bottom}.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0}.table tbody+tbody{border-top:2px solid #ddd}.table-condensed th,.table-condensed td{padding:4px 5px}.table-bordered{border:1px solid #ddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.table-bordered th,.table-bordered td{border-left:1px solid #ddd}.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0}.table-bordered thead:first-child tr:first-child th:first-child,.table-bordered tbody:first-child tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered thead:first-child tr:first-child th:last-child,.table-bordered tbody:first-child tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-radius-topright:4px}.table-bordered thead:last-child tr:last-child th:first-child,.table-bordered tbody:last-child tr:last-child td:first-child,.table-bordered tfoot:last-child tr:last-child td:first-child{-webkit-border-radius:0 0 0 4px;-moz-border-radius:0 0 0 4px;border-radius:0 0 0 4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px}.table-bordered thead:last-child tr:last-child th:last-child,.table-bordered tbody:last-child tr:last-child td:last-child,.table-bordered tfoot:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px}.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topleft:4px}.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-moz-border-right-topleft:4px}.table-striped tbody tr:nth-child(odd) td,.table-striped tbody tr:nth-child(odd) th{background-color:#f9f9f9}.table-hover tbody tr:hover td,.table-hover tbody tr:hover th{background-color:#f5f5f5}table [class*=span],.row-fluid table [class*=span]{display:table-cell;float:none;margin-left:0}table .span1{float:none;width:44px;margin-left:0}table .span2{float:none;width:124px;margin-left:0}table .span3{float:none;width:204px;margin-left:0}table .span4{float:none;width:284px;margin-left:0}table .span5{float:none;width:364px;margin-left:0}table .span6{float:none;width:444px;margin-left:0}table .span7{float:none;width:524px;margin-left:0}table .span8{float:none;width:604px;margin-left:0}table .span9{float:none;width:684px;margin-left:0}table .span10{float:none;width:764px;margin-left:0}table .span11{float:none;width:844px;margin-left:0}table .span12{float:none;width:924px;margin-left:0}table .span13{float:none;width:1004px;margin-left:0}table .span14{float:none;width:1084px;margin-left:0}table .span15{float:none;width:1164px;margin-left:0}table .span16{float:none;width:1244px;margin-left:0}table .span17{float:none;width:1324px;margin-left:0}table .span18{float:none;width:1404px;margin-left:0}table .span19{float:none;width:1484px;margin-left:0}table .span20{float:none;width:1564px;margin-left:0}table .span21{float:none;width:1644px;margin-left:0}table .span22{float:none;width:1724px;margin-left:0}table .span23{float:none;width:1804px;margin-left:0}table .span24{float:none;width:1884px;margin-left:0}.table tbody tr.success td{background-color:#dff0d8}.table tbody tr.error td{background-color:#f2dede}.table tbody tr.info td{background-color:#d9edf7}[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;margin-top:1px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat}.icon-white,.nav>.active>a>[class^="icon-"],.nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png")}.icon-glass{background-position:0 0}.icon-music{background-position:-24px 0}.icon-search{background-position:-48px 0}.icon-envelope{background-position:-72px 0}.icon-heart{background-position:-96px 0}.icon-star{background-position:-120px 0}.icon-star-empty{background-position:-144px 0}.icon-user{background-position:-168px 0}.icon-film{background-position:-192px 0}.icon-th-large{background-position:-216px 0}.icon-th{background-position:-240px 0}.icon-th-list{background-position:-264px 0}.icon-ok{background-position:-288px 0}.icon-remove{background-position:-312px 0}.icon-zoom-in{background-position:-336px 0}.icon-zoom-out{background-position:-360px 0}.icon-off{background-position:-384px 0}.icon-signal{background-position:-408px 0}.icon-cog{background-position:-432px 0}.icon-trash{background-position:-456px 0}.icon-home{background-position:0 -24px}.icon-file{background-position:-24px -24px}.icon-time{background-position:-48px -24px}.icon-road{background-position:-72px -24px}.icon-download-alt{background-position:-96px -24px}.icon-download{background-position:-120px -24px}.icon-upload{background-position:-144px -24px}.icon-inbox{background-position:-168px -24px}.icon-play-circle{background-position:-192px -24px}.icon-repeat{background-position:-216px -24px}.icon-refresh{background-position:-240px -24px}.icon-list-alt{background-position:-264px -24px}.icon-lock{background-position:-287px -24px}.icon-flag{background-position:-312px -24px}.icon-headphones{background-position:-336px -24px}.icon-volume-off{background-position:-360px -24px}.icon-volume-down{background-position:-384px -24px}.icon-volume-up{background-position:-408px -24px}.icon-qrcode{background-position:-432px -24px}.icon-barcode{background-position:-456px -24px}.icon-tag{background-position:0 -48px}.icon-tags{background-position:-25px -48px}.icon-book{background-position:-48px -48px}.icon-bookmark{background-position:-72px -48px}.icon-print{background-position:-96px -48px}.icon-camera{background-position:-120px -48px}.icon-font{background-position:-144px -48px}.icon-bold{background-position:-167px -48px}.icon-italic{background-position:-192px -48px}.icon-text-height{background-position:-216px -48px}.icon-text-width{background-position:-240px -48px}.icon-align-left{background-position:-264px -48px}.icon-align-center{background-position:-288px -48px}.icon-align-right{background-position:-312px -48px}.icon-align-justify{background-position:-336px -48px}.icon-list{background-position:-360px -48px}.icon-indent-left{background-position:-384px -48px}.icon-indent-right{background-position:-408px -48px}.icon-facetime-video{background-position:-432px -48px}.icon-picture{background-position:-456px -48px}.icon-pencil{background-position:0 -72px}.icon-map-marker{background-position:-24px -72px}.icon-adjust{background-position:-48px -72px}.icon-tint{background-position:-72px -72px}.icon-edit{background-position:-96px -72px}.icon-share{background-position:-120px -72px}.icon-check{background-position:-144px -72px}.icon-move{background-position:-168px -72px}.icon-step-backward{background-position:-192px -72px}.icon-fast-backward{background-position:-216px -72px}.icon-backward{background-position:-240px -72px}.icon-play{background-position:-264px -72px}.icon-pause{background-position:-288px -72px}.icon-stop{background-position:-312px -72px}.icon-forward{background-position:-336px -72px}.icon-fast-forward{background-position:-360px -72px}.icon-step-forward{background-position:-384px -72px}.icon-eject{background-position:-408px -72px}.icon-chevron-left{background-position:-432px -72px}.icon-chevron-right{background-position:-456px -72px}.icon-plus-sign{background-position:0 -96px}.icon-minus-sign{background-position:-24px -96px}.icon-remove-sign{background-position:-48px -96px}.icon-ok-sign{background-position:-72px -96px}.icon-question-sign{background-position:-96px -96px}.icon-info-sign{background-position:-120px -96px}.icon-screenshot{background-position:-144px -96px}.icon-remove-circle{background-position:-168px -96px}.icon-ok-circle{background-position:-192px -96px}.icon-ban-circle{background-position:-216px -96px}.icon-arrow-left{background-position:-240px -96px}.icon-arrow-right{background-position:-264px -96px}.icon-arrow-up{background-position:-289px -96px}.icon-arrow-down{background-position:-312px -96px}.icon-share-alt{background-position:-336px -96px}.icon-resize-full{background-position:-360px -96px}.icon-resize-small{background-position:-384px -96px}.icon-plus{background-position:-408px -96px}.icon-minus{background-position:-433px -96px}.icon-asterisk{background-position:-456px -96px}.icon-exclamation-sign{background-position:0 -120px}.icon-gift{background-position:-24px -120px}.icon-leaf{background-position:-48px -120px}.icon-fire{background-position:-72px -120px}.icon-eye-open{background-position:-96px -120px}.icon-eye-close{background-position:-120px -120px}.icon-warning-sign{background-position:-144px -120px}.icon-plane{background-position:-168px -120px}.icon-calendar{background-position:-192px -120px}.icon-random{width:16px;background-position:-216px -120px}.icon-comment{background-position:-240px -120px}.icon-magnet{background-position:-264px -120px}.icon-chevron-up{background-position:-288px -120px}.icon-chevron-down{background-position:-313px -119px}.icon-retweet{background-position:-336px -120px}.icon-shopping-cart{background-position:-360px -120px}.icon-folder-close{background-position:-384px -120px}.icon-folder-open{width:16px;background-position:-408px -120px}.icon-resize-vertical{background-position:-432px -119px}.icon-resize-horizontal{background-position:-456px -118px}.icon-hdd{background-position:0 -144px}.icon-bullhorn{background-position:-24px -144px}.icon-bell{background-position:-48px -144px}.icon-certificate{background-position:-72px -144px}.icon-thumbs-up{background-position:-96px -144px}.icon-thumbs-down{background-position:-120px -144px}.icon-hand-right{background-position:-144px -144px}.icon-hand-left{background-position:-168px -144px}.icon-hand-up{background-position:-192px -144px}.icon-hand-down{background-position:-216px -144px}.icon-circle-arrow-right{background-position:-240px -144px}.icon-circle-arrow-left{background-position:-264px -144px}.icon-circle-arrow-up{background-position:-288px -144px}.icon-circle-arrow-down{background-position:-312px -144px}.icon-globe{background-position:-336px -144px}.icon-wrench{background-position:-360px -144px}.icon-tasks{background-position:-384px -144px}.icon-filter{background-position:-408px -144px}.icon-briefcase{background-position:-432px -144px}.icon-fullscreen{background-position:-456px -144px}.dropup,.dropdown{position:relative}.dropdown-toggle{*margin-bottom:-3px}.dropdown-toggle:active,.open .dropdown-toggle{outline:0}.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000;border-right:4px solid transparent;border-left:4px solid transparent;content:""}.dropdown .caret{margin-top:8px;margin-left:2px}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.dropdown-menu.pull-right{right:0;left:auto}.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.dropdown-menu a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333;white-space:nowrap}.dropdown-menu li>a:hover,.dropdown-menu li>a:focus,.dropdown-submenu:hover>a{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0088cc;background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#0088cc;background-color:#0081c2;background-image:linear-gradient(to bottom,#0088cc,#0087b3);background-image:-moz-linear-gradient(top,#0088cc,#0087b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#0088cc),to(#0087b3));background-image:-webkit-linear-gradient(top,#0088cc,#0087b3);background-image:-o-linear-gradient(top,#0088cc,#0087b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0087b3',GradientType=0)}.dropdown-menu .disabled>a,.dropdown-menu .disabled>a:hover{color:#999}.dropdown-menu .disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.open{*z-index:1000}.open>.dropdown-menu{display:block}.pull-right>.dropdown-menu{right:0;left:auto}.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000;content:"\2191"}.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px}.dropdown-submenu{position:relative}.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px}.dropdown-submenu:hover .dropdown-menu{display:block}.dropdown-submenu>a:after{display:block;float:right;width:0;height:0;margin-top:5px;margin-right:-10px;border-color:transparent;border-left-color:#ccc;border-style:solid;border-width:5px 0 5px 5px;content:" "}.dropdown-submenu:hover>a:after{border-left-color:#fff}.dropdown .dropdown-menu .nav-header{padding-right:20px;padding-left:20px}.typeahead{margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,0.05);box-shadow:inset 0 1px 1px rgba(0,0,0,0.05)}.well blockquote{border-color:#ddd;border-color:rgba(0,0,0,0.15)}.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.fade{opacity:0;-webkit-transition:opacity .15s linear;-moz-transition:opacity .15s linear;-o-transition:opacity .15s linear;transition:opacity .15s linear}.fade.in{opacity:1}.collapse{position:relative;height:0;overflow:hidden;overflow:visible \9;-webkit-transition:height .35s ease;-moz-transition:height .35s ease;-o-transition:height .35s ease;transition:height .35s ease}.collapse.in{height:auto}.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000;text-shadow:0 1px 0 #fff;opacity:.2;filter:alpha(opacity=20)}.close:hover{color:#000;text-decoration:none;cursor:pointer;opacity:.4;filter:alpha(opacity=40)}button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none}.btn{display:inline-block;*display:inline;padding:4px 14px;margin-bottom:0;*margin-left:.3em;font-size:14px;line-height:20px;*line-height:20px;color:#333;text-align:center;text-shadow:0 1px 1px rgba(255,255,255,0.75);vertical-align:middle;cursor:pointer;background-color:#f5f5f5;*background-color:#e6e6e6;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#e6e6e6));background-image:-webkit-linear-gradient(top,#fff,#e6e6e6);background-image:-o-linear-gradient(top,#fff,#e6e6e6);background-image:linear-gradient(to bottom,#fff,#e6e6e6);background-image:-moz-linear-gradient(top,#fff,#e6e6e6);background-repeat:repeat-x;border:1px solid #bbb;*border:0;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-bottom-color:#a2a2a2;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#ffe6e6e6',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);*zoom:1;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn:hover,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333;background-color:#e6e6e6;*background-color:#d9d9d9}.btn:active,.btn.active{background-color:#ccc \9}.btn:first-child{*margin-left:0}.btn:hover{color:#333;text-decoration:none;background-color:#e6e6e6;*background-color:#d9d9d9;background-position:0 -15px;-webkit-transition:background-position .1s linear;-moz-transition:background-position .1s linear;-o-transition:background-position .1s linear;transition:background-position .1s linear}.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px}.btn.active,.btn:active{background-color:#e6e6e6;background-color:#d9d9d9 \9;background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn.disabled,.btn[disabled]{cursor:default;background-color:#e6e6e6;background-image:none;opacity:.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-large{padding:9px 14px;font-size:16px;line-height:normal;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.btn-large [class^="icon-"]{margin-top:2px}.btn-small{padding:3px 9px;font-size:12px;line-height:18px}.btn-small [class^="icon-"]{margin-top:0}.btn-mini{padding:2px 6px;font-size:11px;line-height:16px}.btn-block{display:block;width:100%;padding-right:0;padding-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.btn-block+.btn-block{margin-top:5px}.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255,255,255,0.75)}.btn{border-color:#c5c5c5;border-color:rgba(0,0,0,0.15) rgba(0,0,0,0.15) rgba(0,0,0,0.25)}.btn-primary{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#006dcc;*background-color:#04c;background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#04c));background-image:-webkit-linear-gradient(top,#08c,#04c);background-image:-o-linear-gradient(top,#08c,#04c);background-image:linear-gradient(to bottom,#08c,#04c);background-image:-moz-linear-gradient(top,#08c,#04c);background-repeat:repeat-x;border-color:#04c #04c #002a80;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0044cc',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-primary:hover,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#fff;background-color:#04c;*background-color:#003bb3}.btn-primary:active,.btn-primary.active{background-color:#039 \9}.btn-warning{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#faa732;*background-color:#f89406;background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-repeat:repeat-x;border-color:#f89406 #f89406 #ad6704;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-warning:hover,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#fff;background-color:#f89406;*background-color:#df8505}.btn-warning:active,.btn-warning.active{background-color:#c67605 \9}.btn-danger{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#da4f49;*background-color:#bd362f;background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#bd362f));background-image:-webkit-linear-gradient(top,#ee5f5b,#bd362f);background-image:-o-linear-gradient(top,#ee5f5b,#bd362f);background-image:linear-gradient(to bottom,#ee5f5b,#bd362f);background-image:-moz-linear-gradient(top,#ee5f5b,#bd362f);background-repeat:repeat-x;border-color:#bd362f #bd362f #802420;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffbd362f',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-danger:hover,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#fff;background-color:#bd362f;*background-color:#a9302a}.btn-danger:active,.btn-danger.active{background-color:#942a25 \9}.btn-success{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#5bb75b;*background-color:#51a351;background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#51a351));background-image:-webkit-linear-gradient(top,#62c462,#51a351);background-image:-o-linear-gradient(top,#62c462,#51a351);background-image:linear-gradient(to bottom,#62c462,#51a351);background-image:-moz-linear-gradient(top,#62c462,#51a351);background-repeat:repeat-x;border-color:#51a351 #51a351 #387038;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff51a351',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-success:hover,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#fff;background-color:#51a351;*background-color:#499249}.btn-success:active,.btn-success.active{background-color:#408140 \9}.btn-info{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#49afcd;*background-color:#2f96b4;background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#2f96b4));background-image:-webkit-linear-gradient(top,#5bc0de,#2f96b4);background-image:-o-linear-gradient(top,#5bc0de,#2f96b4);background-image:linear-gradient(to bottom,#5bc0de,#2f96b4);background-image:-moz-linear-gradient(top,#5bc0de,#2f96b4);background-repeat:repeat-x;border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff2f96b4',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-info:hover,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#fff;background-color:#2f96b4;*background-color:#2a85a0}.btn-info:active,.btn-info.active{background-color:#24748c \9}.btn-inverse{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#363636;*background-color:#222;background-image:-webkit-gradient(linear,0 0,0 100%,from(#444),to(#222));background-image:-webkit-linear-gradient(top,#444,#222);background-image:-o-linear-gradient(top,#444,#222);background-image:linear-gradient(to bottom,#444,#222);background-image:-moz-linear-gradient(top,#444,#222);background-repeat:repeat-x;border-color:#222 #222 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff444444',endColorstr='#ff222222',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.btn-inverse:hover,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#fff;background-color:#222;*background-color:#151515}.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9}button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0}button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px}button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px}button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px}.btn-link,.btn-link:active{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.btn-link{color:#08c;cursor:pointer;border-color:transparent;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-link:hover{color:#005580;text-decoration:underline;background-color:transparent}.btn-group{position:relative;*margin-left:.3em;font-size:0;white-space:nowrap}.btn-group:first-child{*margin-left:0}.btn-group+.btn-group{margin-left:5px}.btn-toolbar{margin-top:10px;margin-bottom:10px;font-size:0}.btn-toolbar .btn-group{display:inline-block;*display:inline;*zoom:1}.btn-toolbar .btn+.btn,.btn-toolbar .btn-group+.btn,.btn-toolbar .btn+.btn-group{margin-left:5px}.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group>.btn+.btn{margin-left:-1px}.btn-group>.btn,.btn-group>.dropdown-menu{font-size:14px}.btn-group>.btn-mini{font-size:11px}.btn-group>.btn-small{font-size:12px}.btn-group>.btn-large{font-size:16px}.btn-group>.btn:first-child{margin-left:0;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-bottomleft:4px;-moz-border-radius-topleft:4px}.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-bottomright:4px}.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-bottom-left-radius:6px;border-bottom-left-radius:6px;-webkit-border-top-left-radius:6px;border-top-left-radius:6px;-moz-border-radius-bottomleft:6px;-moz-border-radius-topleft:6px}.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;border-bottom-right-radius:6px;-moz-border-radius-topright:6px;-moz-border-radius-bottomright:6px}.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2}.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0}.btn-group>.btn+.dropdown-toggle{*padding-top:5px;padding-right:8px;*padding-bottom:5px;padding-left:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 1px 0 0 rgba(255,255,255,0.125),inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.05)}.btn-group>.btn-mini+.dropdown-toggle{*padding-top:2px;padding-right:5px;*padding-bottom:2px;padding-left:5px}.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px}.btn-group>.btn-large+.dropdown-toggle{*padding-top:7px;padding-right:12px;*padding-bottom:7px;padding-left:12px}.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05);box-shadow:inset 0 2px 4px rgba(0,0,0,0.15),0 1px 2px rgba(0,0,0,0.05)}.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6}.btn-group.open .btn-primary.dropdown-toggle{background-color:#04c}.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406}.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f}.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351}.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4}.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222}.btn .caret{margin-top:8px;margin-left:0}.btn-mini .caret,.btn-small .caret,.btn-large .caret{margin-top:6px}.btn-large .caret{border-top-width:5px;border-right-width:5px;border-left-width:5px}.dropup .btn-large .caret{border-top:0;border-bottom:5px solid #000}.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#fff;border-bottom-color:#fff}.btn-group-vertical{display:inline-block;*display:inline;*zoom:1}.btn-group-vertical .btn{display:block;float:none;width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.btn-group-vertical .btn+.btn{margin-top:-1px;margin-left:0}.btn-group-vertical .btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.btn-group-vertical .btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.btn-group-vertical .btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0}.btn-group-vertical .btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.alert{padding:8px 35px 8px 14px;margin-bottom:20px;color:#c09853;text-shadow:0 1px 0 rgba(255,255,255,0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.alert h4{margin:0}.alert .close{position:relative;top:-2px;right:-21px;line-height:20px}.alert-success{color:#468847;background-color:#dff0d8;border-color:#d6e9c6}.alert-danger,.alert-error{color:#b94a48;background-color:#f2dede;border-color:#eed3d7}.alert-info{color:#3a87ad;background-color:#d9edf7;border-color:#bce8f1}.alert-block{padding-top:14px;padding-bottom:14px}.alert-block>p,.alert-block>ul{margin-bottom:0}.alert-block p+p{margin-top:5px}.nav{margin-bottom:20px;margin-left:0;list-style:none}.nav>li>a{display:block}.nav>li>a:hover{text-decoration:none;background-color:#eee}.nav>.pull-right{float:right}.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999;text-shadow:0 1px 0 rgba(255,255,255,0.5);text-transform:uppercase}.nav li+.nav-header{margin-top:9px}.nav-list{padding-right:15px;padding-left:15px;margin-bottom:0}.nav-list>li>a,.nav-list .nav-header{margin-right:-15px;margin-left:-15px;text-shadow:0 1px 0 rgba(255,255,255,0.5)}.nav-list>li>a{padding:3px 15px}.nav-list>.active>a,.nav-list>.active>a:hover{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.2);background-color:#08c}.nav-list [class^="icon-"]{margin-right:2px}.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #fff}.nav-tabs,.nav-pills{*zoom:1}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;line-height:0;content:""}.nav-tabs:after,.nav-pills:after{clear:both}.nav-tabs>li,.nav-pills>li{float:left}.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px}.nav-tabs{border-bottom:1px solid #ddd}.nav-tabs>li{margin-bottom:-1px}.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0}.nav-tabs>li>a:hover{border-color:#eee #eee #ddd}.nav-tabs>.active>a,.nav-tabs>.active>a:hover{color:#555;cursor:default;background-color:#fff;border:1px solid #ddd;border-bottom-color:transparent}.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px}.nav-pills>.active>a,.nav-pills>.active>a:hover{color:#fff;background-color:#08c}.nav-stacked>li{float:none}.nav-stacked>li>a{margin-right:0}.nav-tabs.nav-stacked{border-bottom:0}.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;border-top-left-radius:4px;-moz-border-radius-topright:4px;-moz-border-radius-topleft:4px}.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;border-bottom-left-radius:4px;-moz-border-radius-bottomright:4px;-moz-border-radius-bottomleft:4px}.nav-tabs.nav-stacked>li>a:hover{z-index:2;border-color:#ddd}.nav-pills.nav-stacked>li>a{margin-bottom:3px}.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px}.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px}.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.nav .dropdown-toggle .caret{margin-top:6px;border-top-color:#08c;border-bottom-color:#08c}.nav .dropdown-toggle:hover .caret{border-top-color:#005580;border-bottom-color:#005580}.nav-tabs .dropdown-toggle .caret{margin-top:8px}.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.nav>.dropdown.active>a:hover{cursor:pointer}.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover{color:#fff;background-color:#999;border-color:#999}.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret{border-top-color:#fff;border-bottom-color:#fff;opacity:1;filter:alpha(opacity=100)}.tabs-stacked .open>a:hover{border-color:#999}.tabbable{*zoom:1}.tabbable:before,.tabbable:after{display:table;line-height:0;content:""}.tabbable:after{clear:both}.tab-content{overflow:auto}.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0}.tab-content>.tab-pane,.pill-content>.pill-pane{display:none}.tab-content>.active,.pill-content>.active{display:block}.tabs-below>.nav-tabs{border-top:1px solid #ddd}.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0}.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px}.tabs-below>.nav-tabs>li>a:hover{border-top-color:#ddd;border-bottom-color:transparent}.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover{border-color:transparent #ddd #ddd #ddd}.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none}.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px}.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd}.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px}.tabs-left>.nav-tabs>li>a:hover{border-color:#eee #ddd #eee #eee}.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover{border-color:#ddd transparent #ddd #ddd;*border-right-color:#fff}.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd}.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0}.tabs-right>.nav-tabs>li>a:hover{border-color:#eee #eee #eee #ddd}.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover{border-color:#ddd #ddd #ddd transparent;*border-left-color:#fff}.nav>.disabled>a{color:#999}.nav>.disabled>a:hover{text-decoration:none;cursor:default;background-color:transparent}.navbar{*position:relative;*z-index:2;margin-bottom:20px;overflow:visible;color:#555}.navbar-inner{min-height:40px;padding-right:20px;padding-left:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top,#fff,#feead8);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#feead8));background-image:-webkit-linear-gradient(top,#fff,#feead8);background-image:-o-linear-gradient(top,#fff,#feead8);background-image:linear-gradient(to bottom,#fff,#feead8);background-repeat:repeat-x;border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffffffff',endColorstr='#fffeead8',GradientType=0);-webkit-box-shadow:0 1px 4px rgba(0,0,0,0.065);-moz-box-shadow:0 1px 4px rgba(0,0,0,0.065);box-shadow:0 1px 4px rgba(0,0,0,0.065)}.navbar .container{width:auto}.nav-collapse.collapse{height:auto}.navbar .brand{display:block;float:left;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#555;text-shadow:0 1px 0 #fff}.navbar .brand:hover{text-decoration:none}.navbar-text{margin-bottom:0;line-height:40px}.navbar-link{color:#555}.navbar-link:hover{color:#333}.navbar .divider-vertical{height:40px;margin:0 9px;border-right:1px solid #fff;border-left:1px solid #f2f2f2}.navbar .btn,.navbar .btn-group{margin-top:6px}.navbar .btn-group .btn{margin:0}.navbar-form{margin-bottom:0;*zoom:1}.navbar-form:before,.navbar-form:after{display:table;line-height:0;content:""}.navbar-form:after{clear:both}.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px}.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0}.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px}.navbar-form .input-append,.navbar-form .input-prepend{margin-top:6px;white-space:nowrap}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0}.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0}.navbar-search .search-query{padding:4px 14px;margin-bottom:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.navbar-static-top{position:static;width:100%;margin-bottom:0}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner,.navbar-static-top .navbar-inner{border:0}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-right:0;padding-left:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0}.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px}.navbar-fixed-top{top:0}.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.1),0 1px 10px rgba(0,0,0,0.1)}.navbar-fixed-bottom{bottom:0}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1);box-shadow:inset 0 1px 0 rgba(0,0,0,0.1),0 -1px 10px rgba(0,0,0,0.1)}.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0}.navbar .nav.pull-right{float:right}.navbar .nav>li{float:left}.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#555;text-decoration:none;text-shadow:0 1px 0 #fff}.navbar .nav .dropdown-toggle .caret{margin-top:8px}.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{color:#333;text-decoration:none;background-color:transparent}.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);-moz-box-shadow:inset 0 3px 8px rgba(0,0,0,0.125);box-shadow:inset 0 3px 8px rgba(0,0,0,0.125)}.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-right:5px;margin-left:5px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#ededed;*background-color:#e5e5e5;background-image:-webkit-gradient(linear,0 0,0 100%,from(#f2f2f2),to(#e5e5e5));background-image:-webkit-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:-o-linear-gradient(top,#f2f2f2,#e5e5e5);background-image:linear-gradient(to bottom,#f2f2f2,#e5e5e5);background-image:-moz-linear-gradient(top,#f2f2f2,#e5e5e5);background-repeat:repeat-x;border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff2f2f2',endColorstr='#ffe5e5e5',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.075)}.navbar .btn-navbar:hover,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#fff;background-color:#e5e5e5;*background-color:#d9d9d9}.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#ccc \9}.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0,0,0,0.25);-moz-box-shadow:0 1px 0 rgba(0,0,0,0.25);box-shadow:0 1px 0 rgba(0,0,0,0.25)}.btn-navbar .icon-bar+.icon-bar{margin-top:3px}.navbar .nav>li>.dropdown-menu:before{position:absolute;top:-7px;left:9px;display:inline-block;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-left:7px solid transparent;border-bottom-color:rgba(0,0,0,0.2);content:''}.navbar .nav>li>.dropdown-menu:after{position:absolute;top:-6px;left:10px;display:inline-block;border-right:6px solid transparent;border-bottom:6px solid #fff;border-left:6px solid transparent;content:''}.navbar-fixed-bottom .nav>li>.dropdown-menu:before{top:auto;bottom:-7px;border-top:7px solid #ccc;border-bottom:0;border-top-color:rgba(0,0,0,0.2)}.navbar-fixed-bottom .nav>li>.dropdown-menu:after{top:auto;bottom:-6px;border-top:6px solid #fff;border-bottom:0}.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{color:#555;background-color:#e5e5e5}.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555;border-bottom-color:#555}.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{right:0;left:auto}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{right:12px;left:auto}.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{right:13px;left:auto}.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{right:100%;left:auto;margin-right:-1px;margin-left:0;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px}.navbar-inverse{color:#999}.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top,#222,#111);background-image:-webkit-gradient(linear,0 0,0 100%,from(#222),to(#111));background-image:-webkit-linear-gradient(top,#222,#111);background-image:-o-linear-gradient(top,#222,#111);background-image:linear-gradient(to bottom,#222,#111);background-repeat:repeat-x;border-color:#252525;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff222222',endColorstr='#ff111111',GradientType=0)}.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999;text-shadow:0 -1px 0 rgba(0,0,0,0.25)}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover{color:#fff}.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{color:#fff;background-color:transparent}.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#fff;background-color:#111}.navbar-inverse .navbar-link{color:#999}.navbar-inverse .navbar-link:hover{color:#fff}.navbar-inverse .divider-vertical{border-right-color:#222;border-left-color:#111}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{color:#fff;background-color:#111}.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999;border-bottom-color:#999}.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff}.navbar-inverse .navbar-search .search-query{color:#fff;background-color:#515151;border-color:#111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1),0 1px 0 rgba(255,255,255,0.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#ccc}.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333;text-shadow:0 1px 0 #fff;background-color:#fff;border:0;outline:0;-webkit-box-shadow:0 0 3px rgba(0,0,0,0.15);-moz-box-shadow:0 0 3px rgba(0,0,0,0.15);box-shadow:0 0 3px rgba(0,0,0,0.15)}.navbar-inverse .btn-navbar{color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e0e0e;*background-color:#040404;background-image:-webkit-gradient(linear,0 0,0 100%,from(#151515),to(#040404));background-image:-webkit-linear-gradient(top,#151515,#040404);background-image:-o-linear-gradient(top,#151515,#040404);background-image:linear-gradient(to bottom,#151515,#040404);background-image:-moz-linear-gradient(top,#151515,#040404);background-repeat:repeat-x;border-color:#040404 #040404 #000;border-color:rgba(0,0,0,0.1) rgba(0,0,0,0.1) rgba(0,0,0,0.25);filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff151515',endColorstr='#ff040404',GradientType=0);filter:progid:dximagetransform.microsoft.gradient(enabled=false)}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#fff;background-color:#040404;*background-color:#000}.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000 \9}.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.breadcrumb li{display:inline-block;*display:inline;text-shadow:0 1px 0 #fff;*zoom:1}.breadcrumb .divider{padding:0 5px;color:#ccc}.breadcrumb .active{color:#999}.pagination{height:40px;margin:20px 0}.pagination ul{display:inline-block;*display:inline;margin-bottom:0;margin-left:0;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;*zoom:1;-webkit-box-shadow:0 1px 2px rgba(0,0,0,0.05);-moz-box-shadow:0 1px 2px rgba(0,0,0,0.05);box-shadow:0 1px 2px rgba(0,0,0,0.05)}.pagination li{display:inline}.pagination a,.pagination span{float:left;padding:0 14px;line-height:38px;text-decoration:none;background-color:#fff;border:1px solid #ddd;border-left-width:0}.pagination a:hover,.pagination .active a,.pagination .active span{background-color:#f5f5f5}.pagination .active a,.pagination .active span{color:#999;cursor:default}.pagination .disabled span,.pagination .disabled a,.pagination .disabled a:hover{color:#999;cursor:default;background-color:transparent}.pagination li:first-child a,.pagination li:first-child span{border-left-width:1px;-webkit-border-radius:3px 0 0 3px;-moz-border-radius:3px 0 0 3px;border-radius:3px 0 0 3px}.pagination li:last-child a,.pagination li:last-child span{-webkit-border-radius:0 3px 3px 0;-moz-border-radius:0 3px 3px 0;border-radius:0 3px 3px 0}.pagination-centered{text-align:center}.pagination-right{text-align:right}.pager{margin:20px 0;text-align:center;list-style:none;*zoom:1}.pager:before,.pager:after{display:table;line-height:0;content:""}.pager:after{clear:both}.pager li{display:inline}.pager a{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px}.pager a:hover{text-decoration:none;background-color:#f5f5f5}.pager .next a{float:right}.pager .previous a{float:left}.pager .disabled a,.pager .disabled a:hover{color:#999;cursor:default;background-color:#fff}.modal-open .dropdown-menu{z-index:2050}.modal-open .dropdown.open{*z-index:2050}.modal-open .popover{z-index:2060}.modal-open .tooltip{z-index:2080}.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000}.modal-backdrop.fade{opacity:0}.modal-backdrop,.modal-backdrop.fade.in{opacity:.8;filter:alpha(opacity=80)}.modal{position:fixed;top:50%;left:50%;z-index:1050;width:560px;margin:-250px 0 0 -280px;overflow:auto;background-color:#fff;border:1px solid #999;border:1px solid rgba(0,0,0,0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0,0,0,0.3);-moz-box-shadow:0 3px 7px rgba(0,0,0,0.3);box-shadow:0 3px 7px rgba(0,0,0,0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box}.modal.fade{top:-25%;-webkit-transition:opacity .3s linear,top .3s ease-out;-moz-transition:opacity .3s linear,top .3s ease-out;-o-transition:opacity .3s linear,top .3s ease-out;transition:opacity .3s linear,top .3s ease-out}.modal.fade.in{top:50%}.modal-header{padding:9px 15px;border-bottom:1px solid #eee}.modal-header .close{margin-top:2px}.modal-header h3{margin:0;line-height:30px}.modal-body{max-height:400px;padding:15px;overflow-y:auto}.modal-form{margin-bottom:0}.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;*zoom:1;-webkit-box-shadow:inset 0 1px 0 #fff;-moz-box-shadow:inset 0 1px 0 #fff;box-shadow:inset 0 1px 0 #fff}.modal-footer:before,.modal-footer:after{display:table;line-height:0;content:""}.modal-footer:after{clear:both}.modal-footer .btn+.btn{margin-bottom:0;margin-left:5px}.modal-footer .btn-group .btn+.btn{margin-left:-1px}.tooltip{position:absolute;z-index:1030;display:block;padding:5px;font-size:11px;opacity:0;filter:alpha(opacity=0);visibility:visible}.tooltip.in{opacity:.8;filter:alpha(opacity=80)}.tooltip.top{margin-top:-3px}.tooltip.right{margin-left:3px}.tooltip.bottom{margin-top:3px}.tooltip.left{margin-left:-3px}.tooltip-inner{max-width:200px;padding:3px 8px;color:#fff;text-align:center;text-decoration:none;background-color:#000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid}.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-top-color:#000;border-width:5px 5px 0}.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-right-color:#000;border-width:5px 5px 5px 0}.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-left-color:#000;border-width:5px 0 5px 5px}.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-bottom-color:#000;border-width:0 5px 5px}.popover{position:absolute;top:0;left:0;z-index:1010;display:none;width:236px;padding:1px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0,0,0,0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0,0,0,0.2);-moz-box-shadow:0 5px 10px rgba(0,0,0,0.2);box-shadow:0 5px 10px rgba(0,0,0,0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box}.popover.top{margin-bottom:10px}.popover.right{margin-left:10px}.popover.bottom{margin-top:10px}.popover.left{margin-right:10px}.popover-title{padding:8px 14px;margin:0;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0}.popover-content{padding:9px 14px}.popover-content p,.popover-content ul,.popover-content ol{margin-bottom:0}.popover .arrow,.popover .arrow:after{position:absolute;display:inline-block;width:0;height:0;border-color:transparent;border-style:solid}.popover .arrow:after{z-index:-1;content:""}.popover.top .arrow{bottom:-10px;left:50%;margin-left:-10px;border-top-color:#fff;border-width:10px 10px 0}.popover.top .arrow:after{bottom:-1px;left:-11px;border-top-color:rgba(0,0,0,0.25);border-width:11px 11px 0}.popover.right .arrow{top:50%;left:-10px;margin-top:-10px;border-right-color:#fff;border-width:10px 10px 10px 0}.popover.right .arrow:after{bottom:-11px;left:-1px;border-right-color:rgba(0,0,0,0.25);border-width:11px 11px 11px 0}.popover.bottom .arrow{top:-10px;left:50%;margin-left:-10px;border-bottom-color:#fff;border-width:0 10px 10px}.popover.bottom .arrow:after{top:-1px;left:-11px;border-bottom-color:rgba(0,0,0,0.25);border-width:0 11px 11px}.popover.left .arrow{top:50%;right:-10px;margin-top:-10px;border-left-color:#fff;border-width:10px 0 10px 10px}.popover.left .arrow:after{right:-1px;bottom:-11px;border-left-color:rgba(0,0,0,0.25);border-width:11px 0 11px 11px}.thumbnails{margin-left:-20px;list-style:none;*zoom:1}.thumbnails:before,.thumbnails:after{display:table;line-height:0;content:""}.thumbnails:after{clear:both}.row-fluid .thumbnails{margin-left:0}.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px}.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0,0,0,0.055);-moz-box-shadow:0 1px 3px rgba(0,0,0,0.055);box-shadow:0 1px 3px rgba(0,0,0,0.055);-webkit-transition:all .2s ease-in-out;-moz-transition:all .2s ease-in-out;-o-transition:all .2s ease-in-out;transition:all .2s ease-in-out}a.thumbnail:hover{border-color:#08c;-webkit-box-shadow:0 1px 4px rgba(0,105,214,0.25);-moz-box-shadow:0 1px 4px rgba(0,105,214,0.25);box-shadow:0 1px 4px rgba(0,105,214,0.25)}.thumbnail>img{display:block;max-width:100%;margin-right:auto;margin-left:auto}.thumbnail .caption{padding:9px;color:#555}.label,.badge{font-size:11.844px;font-weight:bold;line-height:14px;color:#fff;text-shadow:0 -1px 0 rgba(0,0,0,0.25);white-space:nowrap;vertical-align:baseline;background-color:#999}.label{padding:1px 4px 2px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.badge{padding:1px 9px 2px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px}a.label:hover,a.badge:hover{color:#fff;text-decoration:none;cursor:pointer}.label-important,.badge-important{background-color:#b94a48}.label-important[href],.badge-important[href]{background-color:#953b39}.label-warning,.badge-warning{background-color:#f89406}.label-warning[href],.badge-warning[href]{background-color:#c67605}.label-success,.badge-success{background-color:#468847}.label-success[href],.badge-success[href]{background-color:#356635}.label-info,.badge-info{background-color:#3a87ad}.label-info[href],.badge-info[href]{background-color:#2d6987}.label-inverse,.badge-inverse{background-color:#333}.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a}.btn .label,.btn .badge{position:relative;top:-1px}.btn-mini .label,.btn-mini .badge{top:0}@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}@-o-keyframes progress-bar-stripes{from{background-position:0 0}to{background-position:40px 0}}@keyframes progress-bar-stripes{from{background-position:40px 0}to{background-position:0 0}}.progress{height:20px;margin-bottom:20px;overflow:hidden;background-color:#f7f7f7;background-image:-moz-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#f5f5f5),to(#f9f9f9));background-image:-webkit-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:-o-linear-gradient(top,#f5f5f5,#f9f9f9);background-image:linear-gradient(to bottom,#f5f5f5,#f9f9f9);background-repeat:repeat-x;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fff5f5f5',endColorstr='#fff9f9f9',GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,0.1);box-shadow:inset 0 1px 2px rgba(0,0,0,0.1)}.progress .bar{float:left;width:0;height:100%;font-size:12px;color:#fff;text-align:center;text-shadow:0 -1px 0 rgba(0,0,0,0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top,#149bdf,#0480be);background-image:-webkit-gradient(linear,0 0,0 100%,from(#149bdf),to(#0480be));background-image:-webkit-linear-gradient(top,#149bdf,#0480be);background-image:-o-linear-gradient(top,#149bdf,#0480be);background-image:linear-gradient(to bottom,#149bdf,#0480be);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff149bdf',endColorstr='#ff0480be',GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 0 -1px 0 rgba(0,0,0,0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width .6s ease;-moz-transition:width .6s ease;-o-transition:width .6s ease;transition:width .6s ease}.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15);box-shadow:inset 1px 0 0 rgba(0,0,0,0.15),inset 0 -1px 0 rgba(0,0,0,0.15)}.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px}.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite}.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top,#ee5f5b,#c43c35);background-image:-webkit-gradient(linear,0 0,0 100%,from(#ee5f5b),to(#c43c35));background-image:-webkit-linear-gradient(top,#ee5f5b,#c43c35);background-image:-o-linear-gradient(top,#ee5f5b,#c43c35);background-image:linear-gradient(to bottom,#ee5f5b,#c43c35);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ffee5f5b',endColorstr='#ffc43c35',GradientType=0)}.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top,#62c462,#57a957);background-image:-webkit-gradient(linear,0 0,0 100%,from(#62c462),to(#57a957));background-image:-webkit-linear-gradient(top,#62c462,#57a957);background-image:-o-linear-gradient(top,#62c462,#57a957);background-image:linear-gradient(to bottom,#62c462,#57a957);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff62c462',endColorstr='#ff57a957',GradientType=0)}.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top,#5bc0de,#339bb9);background-image:-webkit-gradient(linear,0 0,0 100%,from(#5bc0de),to(#339bb9));background-image:-webkit-linear-gradient(top,#5bc0de,#339bb9);background-image:-o-linear-gradient(top,#5bc0de,#339bb9);background-image:linear-gradient(to bottom,#5bc0de,#339bb9);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff5bc0de',endColorstr='#ff339bb9',GradientType=0)}.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top,#fbb450,#f89406);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fbb450),to(#f89406));background-image:-webkit-linear-gradient(top,#fbb450,#f89406);background-image:-o-linear-gradient(top,#fbb450,#f89406);background-image:linear-gradient(to bottom,#fbb450,#f89406);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#fffbb450',endColorstr='#fff89406',GradientType=0)}.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear,0 100%,100% 0,color-stop(0.25,rgba(255,255,255,0.15)),color-stop(0.25,transparent),color-stop(0.5,transparent),color-stop(0.5,rgba(255,255,255,0.15)),color-stop(0.75,rgba(255,255,255,0.15)),color-stop(0.75,transparent),to(transparent));background-image:-webkit-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-moz-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:-o-linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent);background-image:linear-gradient(45deg,rgba(255,255,255,0.15) 25%,transparent 25%,transparent 50%,rgba(255,255,255,0.15) 50%,rgba(255,255,255,0.15) 75%,transparent 75%,transparent)}.accordion{margin-bottom:20px}.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.accordion-heading{border-bottom:0}.accordion-heading .accordion-toggle{display:block;padding:8px 15px}.accordion-toggle{cursor:pointer}.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5}.carousel{position:relative;margin-bottom:20px;line-height:1}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel .item{position:relative;display:none;-webkit-transition:.6s ease-in-out left;-moz-transition:.6s ease-in-out left;-o-transition:.6s ease-in-out left;transition:.6s ease-in-out left}.carousel .item>img{display:block;line-height:1}.carousel .active,.carousel .next,.carousel .prev{display:block}.carousel .active{left:0}.carousel .next,.carousel .prev{position:absolute;top:0;width:100%}.carousel .next{left:100%}.carousel .prev{left:-100%}.carousel .next.left,.carousel .prev.right{left:0}.carousel .active.left{left:-100%}.carousel .active.right{left:100%}.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#fff;text-align:center;background:#222;border:3px solid #fff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:.5;filter:alpha(opacity=50)}.carousel-control.right{right:15px;left:auto}.carousel-control:hover{color:#fff;text-decoration:none;opacity:.9;filter:alpha(opacity=90)}.carousel-caption{position:absolute;right:0;bottom:0;left:0;padding:15px;background:#333;background:rgba(0,0,0,0.75)}.carousel-caption h4,.carousel-caption p{line-height:20px;color:#fff}.carousel-caption h4{margin:0 0 5px}.carousel-caption p{margin-bottom:0}.hero-unit{padding:60px;margin-bottom:30px;background-color:#eee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;letter-spacing:-1px;color:inherit}.hero-unit p{font-size:18px;font-weight:200;line-height:30px;color:inherit}.pull-right{float:right}.pull-left{float:left}.hide{display:none}.show{display:block}.invisible{visibility:hidden}.affix{position:fixed}
diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md
index b33af2cf24..de6a2b0a43 100644
--- a/docs/hadoop-third-party-distributions.md
+++ b/docs/hadoop-third-party-distributions.md
@@ -10,7 +10,7 @@ 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):
+[set the SPARK_HADOOP_VERSION flag](index.html#a-note-about-hadoop-versions):
SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly
@@ -40,6 +40,7 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors.
<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>
+ <tr><td>HDP 2.0</td><td>2.2.0</td></tr>
</table>
</td>
</tr>
diff --git a/docs/index.md b/docs/index.md
index bd386a8a8f..86d574daaa 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -24,9 +24,9 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_VE
# 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
-(the `run-example` script sets up the appropriate paths and launches that program).
-For example, try `./run-example org.apache.spark.examples.SparkPi local`.
+To run one of the samples, use `./bin/run-example <class> <params>` in the top-level Spark directory
+(the `bin/run-example` script sets up the appropriate paths and launches that program).
+For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`.
Each example prints usage help when run with no parameters.
Note that all of the sample programs take a `<master>` parameter specifying the cluster URL
@@ -34,8 +34,8 @@ 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, 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.
+Finally, you can run Spark interactively through modified versions of the Scala shell (`./bin/spark-shell`) or
+Python interpreter (`./bin/pyspark`). These are a great way to learn the framework.
# Launching on a Cluster
@@ -56,14 +56,16 @@ Hadoop, you must build Spark against the same version that your cluster uses.
By default, Spark links to Hadoop 1.0.4. You can change this by setting the
`SPARK_HADOOP_VERSION` variable when compiling:
- SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly
+ SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly
-In addition, if you wish to run Spark on [YARN](running-on-yarn.md), set
+In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set
`SPARK_YARN` to `true`:
SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly
-(Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.)
+Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`.
+
+For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.html). This is needed because Hadoop 2.2 has non backwards compatible API changes.
# Where to Go from Here
diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md
index 53085cc671..07732fa122 100644
--- a/docs/java-programming-guide.md
+++ b/docs/java-programming-guide.md
@@ -190,9 +190,9 @@ We hope to generate documentation with Java-style syntax in the future.
Spark includes several sample programs using the Java API in
[`examples/src/main/java`](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the
-`run-example` script included in Spark; for example:
+`bin/run-example` script included in Spark; for example:
- ./run-example org.apache.spark.examples.JavaWordCount
+ ./bin/run-example org.apache.spark.examples.JavaWordCount
Each example program prints usage help when run
without any arguments.
diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md
index d304c5497b..df2faa5e41 100644
--- a/docs/job-scheduling.md
+++ b/docs/job-scheduling.md
@@ -32,12 +32,11 @@ 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.
+ the number of nodes an application uses by setting the `spark.cores.max` configuration property in it,
+ or change the default for applications that don't set this setting through `spark.deploy.defaultCores`.
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`,
+* **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration 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
@@ -78,11 +77,13 @@ of cluster resources. This means that short jobs submitted while a long job is r
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
+To enable the fair scheduler, simply set the `spark.scheduler.mode` property to `FAIR` when configuring
a SparkContext:
{% highlight scala %}
-System.setProperty("spark.scheduler.mode", "FAIR")
+val conf = new SparkConf().setMaster(...).setAppName(...)
+conf.set("spark.scheduler.mode", "FAIR")
+val sc = new SparkContext(conf)
{% endhighlight %}
## Fair Scheduler Pools
@@ -91,15 +92,15 @@ The fair scheduler also supports grouping jobs into _pools_, and setting differe
(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).
+[Hadoop Fair Scheduler](http://hadoop.apache.org/docs/current/hadoop-yarn/hadoop-yarn-site/FairScheduler.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")
+// Assuming sc is your SparkContext variable
+sc.setLocalProperty("spark.scheduler.pool", "pool1")
{% endhighlight %}
After setting this local property, _all_ jobs submitted within this thread (by calls in this thread
@@ -108,7 +109,7 @@ it easy to have a thread run multiple jobs on behalf of the same user. If you'd
pool that a thread is associated with, simply call:
{% highlight scala %}
-context.setLocalProperty("spark.scheduler.pool", null)
+sc.setLocalProperty("spark.scheduler.pool", null)
{% endhighlight %}
## Default Behavior of Pools
@@ -138,10 +139,11 @@ properties:
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:
+and setting a `spark.scheduler.allocation.file` property in your
+[SparkConf](configuration.html#spark-properties).
{% highlight scala %}
-System.setProperty("spark.scheduler.allocation.file", "/path/to/file")
+conf.set("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
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index c1ff9c417c..45ee166688 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -3,6 +3,9 @@ layout: global
title: Machine Learning Library (MLlib)
---
+* Table of contents
+{:toc}
+
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,
@@ -39,57 +42,6 @@ underlying gradient descent primitive (described
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)
@@ -121,14 +73,14 @@ 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
+[k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of
+the most commonly used clustering algorithms that clusters the data points into
+predfined number of 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.
+* *k* is the number of desired clusters.
* *maxIterations* is the maximum number of iterations to run.
* *initializationMode* specifies either random initialization or
initialization via k-means\|\|.
@@ -169,7 +121,7 @@ the entries in the user-item matrix as *explicit* preferences given by the user
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).
+[Collaborative Filtering for Implicit Feedback Datasets](http://www2.research.att.com/~yifanhu/PUB/cf.pdf).
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.
@@ -210,3 +162,269 @@ at each iteration.
Available algorithms for gradient descent:
* [GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent)
+
+# Using MLLib in Scala
+
+Following code snippets can be executed in `spark-shell`.
+
+## Binary Classification
+
+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 to build the model
+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("Training Error = " + 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 %}
+
+## Linear Regression
+The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The
+example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We
+compute the Mean Squared Error at the end to evaluate
+[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit)
+
+{% highlight scala %}
+import org.apache.spark.mllib.regression.LinearRegressionWithSGD
+import org.apache.spark.mllib.regression.LabeledPoint
+
+// Load and parse the data
+val data = sc.textFile("mllib/data/ridge-data/lpsa.data")
+val parsedData = data.map { line =>
+ val parts = line.split(',')
+ LabeledPoint(parts(0).toDouble, parts(1).split(' ').map(x => x.toDouble).toArray)
+}
+
+// Building the model
+val numIterations = 20
+val model = LinearRegressionWithSGD.train(parsedData, numIterations)
+
+// Evaluate model on training examples and compute training error
+val valuesAndPreds = parsedData.map { point =>
+ val prediction = model.predict(point.features)
+ (point.label, prediction)
+}
+val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count
+println("training Mean Squared Error = " + MSE)
+{% endhighlight %}
+
+
+Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training
+[Mean Squared Errors](http://en.wikipedia.org/wiki/Mean_squared_error).
+
+## Clustering
+In the following example after loading and parsing data, we use the KMeans object to cluster the data
+into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within
+Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the
+optimal *k* is usually one where there is an "elbow" in the WSSSE graph.
+
+{% highlight scala %}
+import org.apache.spark.mllib.clustering.KMeans
+
+// Load and parse the data
+val data = sc.textFile("kmeans_data.txt")
+val parsedData = data.map( _.split(' ').map(_.toDouble))
+
+// Cluster the data into two classes using KMeans
+val numIterations = 20
+val numClusters = 2
+val clusters = KMeans.train(parsedData, numClusters, numIterations)
+
+// Evaluate clustering by computing Within Set Sum of Squared Errors
+val WSSSE = clusters.computeCost(parsedData)
+println("Within Set Sum of Squared Errors = " + WSSSE)
+{% endhighlight %}
+
+
+## Collaborative Filtering
+In the following example we load rating data. Each row consists of a user, a product and a rating.
+We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation
+model by measuring the Mean Squared Error of rating prediction.
+
+{% highlight scala %}
+import org.apache.spark.mllib.recommendation.ALS
+import org.apache.spark.mllib.recommendation.Rating
+
+// Load and parse the data
+val data = sc.textFile("mllib/data/als/test.data")
+val ratings = data.map(_.split(',') match {
+ case Array(user, item, rate) => Rating(user.toInt, item.toInt, rate.toDouble)
+})
+
+// Build the recommendation model using ALS
+val numIterations = 20
+val model = ALS.train(ratings, 1, 20, 0.01)
+
+// Evaluate the model on rating data
+val usersProducts = ratings.map{ case Rating(user, product, rate) => (user, product)}
+val predictions = model.predict(usersProducts).map{
+ case Rating(user, product, rate) => ((user, product), rate)
+}
+val ratesAndPreds = ratings.map{
+ case Rating(user, product, rate) => ((user, product), rate)
+}.join(predictions)
+val MSE = ratesAndPreds.map{
+ case ((user, product), (r1, r2)) => math.pow((r1- r2), 2)
+}.reduce(_ + _)/ratesAndPreds.count
+println("Mean Squared Error = " + MSE)
+{% endhighlight %}
+
+If the rating matrix is derived from other source of information (i.e., it is inferred from
+other signals), you can use the trainImplicit method to get better results.
+
+{% highlight scala %}
+val model = ALS.trainImplicit(ratings, 1, 20, 0.01)
+{% endhighlight %}
+
+# Using MLLib in Python
+Following examples can be tested in the PySpark shell.
+
+## Binary Classification
+The following example shows how to load a sample dataset, build Logistic Regression model,
+and make predictions with the resulting model to compute the training error.
+
+{% highlight python %}
+from pyspark.mllib.classification import LogisticRegressionWithSGD
+from numpy import array
+
+# Load and parse the data
+data = sc.textFile("mllib/data/sample_svm_data.txt")
+parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')]))
+model = LogisticRegressionWithSGD.train(sc, parsedData)
+
+# Build the model
+labelsAndPreds = parsedData.map(lambda point: (int(point.item(0)),
+ model.predict(point.take(range(1, point.size)))))
+
+# Evaluating the model on training data
+trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count())
+print("Training Error = " + str(trainErr))
+{% endhighlight %}
+
+## Linear Regression
+The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The
+example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We
+compute the Mean Squared Error at the end to evaluate
+[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit)
+
+{% highlight python %}
+from pyspark.mllib.regression import LinearRegressionWithSGD
+from numpy import array
+
+# Load and parse the data
+data = sc.textFile("mllib/data/ridge-data/lpsa.data")
+parsedData = data.map(lambda line: array([float(x) for x in line.replace(',', ' ').split(' ')]))
+
+# Build the model
+model = LinearRegressionWithSGD.train(sc, parsedData)
+
+# Evaluate the model on training data
+valuesAndPreds = parsedData.map(lambda point: (point.item(0),
+ model.predict(point.take(range(1, point.size)))))
+MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y)/valuesAndPreds.count()
+print("Mean Squared Error = " + str(MSE))
+{% endhighlight %}
+
+
+## Clustering
+In the following example after loading and parsing data, we use the KMeans object to cluster the data
+into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within
+Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the
+optimal *k* is usually one where there is an "elbow" in the WSSSE graph.
+
+{% highlight python %}
+from pyspark.mllib.clustering import KMeans
+from numpy import array
+from math import sqrt
+
+# Load and parse the data
+data = sc.textFile("kmeans_data.txt")
+parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')]))
+
+# Build the model (cluster the data)
+clusters = KMeans.train(sc, parsedData, 2, maxIterations=10,
+ runs=30, initialization_mode="random")
+
+# Evaluate clustering by computing Within Set Sum of Squared Errors
+def error(point):
+ center = clusters.centers[clusters.predict(point)]
+ return sqrt(sum([x**2 for x in (point - center)]))
+
+WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y)
+print("Within Set Sum of Squared Error = " + str(WSSSE))
+{% endhighlight %}
+
+Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training Mean Squared
+Errors.
+
+## Collaborative Filtering
+In the following example we load rating data. Each row consists of a user, a product and a rating.
+We use the default ALS.train() method which assumes ratings are explicit. We evaluate the
+recommendation by measuring the Mean Squared Error of rating prediction.
+
+{% highlight python %}
+from pyspark.mllib.recommendation import ALS
+from numpy import array
+
+# Load and parse the data
+data = sc.textFile("mllib/data/als/test.data")
+ratings = data.map(lambda line: array([float(x) for x in line.split(',')]))
+
+# Build the recommendation model using Alternating Least Squares
+model = ALS.train(sc, ratings, 1, 20)
+
+# Evaluate the model on training data
+testdata = ratings.map(lambda p: (int(p[0]), int(p[1])))
+predictions = model.predictAll(testdata).map(lambda r: ((r[0], r[1]), r[2]))
+ratesAndPreds = ratings.map(lambda r: ((r[0], r[1]), r[2])).join(predictions)
+MSE = ratesAndPreds.map(lambda r: (r[1][0] - r[1][1])**2).reduce(lambda x, y: x + y)/ratesAndPreds.count()
+print("Mean Squared Error = " + str(MSE))
+{% endhighlight %}
+
+If the rating matrix is derived from other source of information (i.e., it is inferred from other
+signals), you can use the trainImplicit method to get better results.
+
+{% highlight python %}
+# Build the recommendation model using Alternating Least Squares based on implicit ratings
+model = ALS.trainImplicit(sc, ratings, 1, 20)
+{% endhighlight %}
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 5ed0474477..0d5eb7065e 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -32,7 +32,8 @@ Spark has a configurable metrics system based on the
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
+`spark.metrics.conf` [configuration property](configuration.html#spark-properties).
+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:
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 55e39b1de1..dc187b3efe 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -47,7 +47,7 @@ PySpark will automatically ship these functions to workers, along with any objec
Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers.
The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers.
-In addition, PySpark fully supports interactive use---simply run `./pyspark` to launch an interactive shell.
+In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell.
# Installing and Configuring PySpark
@@ -60,17 +60,17 @@ By default, PySpark requires `python` to be available on the system `PATH` and u
All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported.
-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`.
+Standalone PySpark applications should be run using the `bin/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 `bin/pyspark` package to the `PYTHONPATH`.
# Interactive Use
-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:
+The `bin/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
-$ ./pyspark
+$ ./bin/pyspark
{% endhighlight %}
The Python shell can be used explore data interactively and is a simple way to learn the API:
@@ -82,35 +82,35 @@ 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 applications locally on a single core.
+By default, the `bin/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):
+For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html):
{% highlight bash %}
-$ MASTER=spark://IP:PORT ./pyspark
+$ MASTER=spark://IP:PORT ./bin/pyspark
{% endhighlight %}
Or, to use four cores on the local machine:
{% highlight bash %}
-$ MASTER=local[4] ./pyspark
+$ MASTER=local[4] ./bin/pyspark
{% endhighlight %}
## IPython
It is also possible to launch PySpark in [IPython](http://ipython.org), the enhanced Python interpreter.
-To do this, set the `IPYTHON` variable to `1` when running `pyspark`:
+To do this, set the `IPYTHON` variable to `1` when running `bin/pyspark`:
{% highlight bash %}
-$ IPYTHON=1 ./pyspark
+$ IPYTHON=1 ./bin/pyspark
{% endhighlight %}
Alternatively, you can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch
the [IPython Notebook](http://ipython.org/notebook.html) with PyLab graphing support:
{% highlight bash %}
-$ IPYTHON_OPTS="notebook --pylab inline" ./pyspark
+$ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark
{% endhighlight %}
IPython also works on a cluster or on multiple cores if you set the `MASTER` environment variable.
@@ -118,7 +118,7 @@ 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`.
+PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`.
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:
@@ -131,15 +131,16 @@ sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg
Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines.
Code dependencies can be added to an existing SparkContext using its `addPyFile()` method.
-You can set [system properties](configuration.html#system-properties)
-using `SparkContext.setSystemProperty()` class method *before*
-instantiating SparkContext. For example, to set the amount of memory
-per executor process:
+You can set [configuration properties](configuration.html#spark-properties) by passing a
+[SparkConf](api/pyspark/pyspark.conf.SparkConf-class.html) object to SparkContext:
{% highlight python %}
-from pyspark import SparkContext
-SparkContext.setSystemProperty('spark.executor.memory', '2g')
-sc = SparkContext("local", "App Name")
+from pyspark import SparkConf, SparkContext
+conf = (SparkConf()
+ .setMaster("local")
+ .setAppName("My app")
+ .set("spark.executor.memory", "1g"))
+sc = SparkContext(conf = conf)
{% endhighlight %}
# API Docs
@@ -152,6 +153,6 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc
PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples).
You can run them by passing the files to `pyspark`; e.g.:
- ./pyspark python/examples/wordcount.py
+ ./bin/pyspark python/examples/wordcount.py
Each program prints usage help when run without arguments.
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 8f782db5b8..153081bdaa 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -20,7 +20,7 @@ $ sbt/sbt assembly
## Basics
Spark's interactive shell provides a simple way to learn the API, as well as a powerful tool to analyze datasets interactively.
-Start the shell by running `./spark-shell` in the Spark directory.
+Start the shell by running `./bin/spark-shell` in the Spark directory.
Spark's primary abstraction is a distributed collection of items called a Resilient Distributed Dataset (RDD). RDDs can be created from Hadoop InputFormats (such as HDFS files) or by transforming other RDDs. Let's make a new RDD from the text of the README file in the Spark source directory:
@@ -99,7 +99,7 @@ scala> linesWithSpark.count()
res9: Long = 15
{% endhighlight %}
-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).
+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 `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
# 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.
@@ -124,7 +124,7 @@ object SimpleApp {
}
{% endhighlight %}
-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 program just 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 `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.
+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/sbt run` to execute our program.
{% highlight bash %}
$ find .
@@ -157,8 +157,8 @@ $ find .
./src/main/scala
./src/main/scala/SimpleApp.scala
-$ sbt package
-$ sbt run
+$ sbt/sbt package
+$ sbt/sbt run
...
Lines with a: 46, Lines with b: 23
{% endhighlight %}
@@ -193,7 +193,7 @@ public class SimpleApp {
}
{% endhighlight %}
-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.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a text 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 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.
@@ -224,11 +224,11 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a
If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS:
{% highlight xml %}
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-client</artifactId>
- <version>...</version>
- </dependency>
+<dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-client</artifactId>
+ <version>...</version>
+</dependency>
{% endhighlight %}
We lay out these files according to the canonical Maven directory structure:
@@ -270,18 +270,18 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs)
{% endhighlight %}
-This program simply counts the number of lines containing 'a' and the number containing 'b' in a system log file.
+This program just counts the number of lines containing 'a' and the number containing 'b' in a text 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 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 application using the `pyspark` script:
+We can run this application using the `bin/pyspark` script:
{% highlight python %}
$ cd $SPARK_HOME
-$ ./pyspark SimpleApp.py
+$ ./bin/pyspark SimpleApp.py
...
Lines with a: 46, Lines with b: 23
{% endhighlight python %}
@@ -301,14 +301,38 @@ assembly jar (or "uber" jar) containing your code and its dependencies. Both
have assembly plugins. When creating assembly jars, list Spark
itself as a `provided` dependency; it need not be bundled since it is
already present on the slaves. Once you have an assembled jar,
-add it to the SparkContext as shown here. It is also possible to submit
-your dependent jars one-by-one when creating a SparkContext.
+add it to the SparkContext as shown here. It is also possible to add
+your dependent jars one-by-one using the `addJar` method of `SparkContext`.
+
+For Python, you can use the `pyFiles` argument of SparkContext
+or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed.
### Setting Configuration Options
-Spark includes several configuration options which influence the behavior
-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.
+Spark includes several [configuration options](configuration.html#spark-properties)
+that influence the behavior of your application.
+These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+object and passing it to the SparkContext constructor.
+For example, in Java and Scala, you can do:
+
+{% highlight scala %}
+import org.apache.spark.{SparkConf, SparkContext}
+val conf = new SparkConf()
+ .setMaster("local")
+ .setAppName("My application")
+ .set("spark.executor.memory", "1g")
+val sc = new SparkContext(conf)
+{% endhighlight %}
+
+Or in Python:
+
+{% highlight scala %}
+from pyspark import SparkConf, SparkContext
+conf = SparkConf()
+conf.setMaster("local")
+conf.setAppName("My application")
+conf.set("spark.executor.memory", "1g"))
+sc = SparkContext(conf = conf)
+{% endhighlight %}
### Accessing Hadoop Filesystems
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index 322ff585f1..68259f0cb8 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -15,15 +15,16 @@ Spark can run on clusters managed by [Apache Mesos](http://mesos.apache.org/). F
* `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:
+8. To run a Spark application against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:PORT` as the master URL. In addition, you'll need to set the `spark.executor.uri` property. For example:
{% highlight scala %}
-System.setProperty("spark.executor.uri", "<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>")
-val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
+val conf = new SparkConf()
+ .setMaster("mesos://HOST:5050")
+ .setAppName("My app")
+ .set("spark.executor.uri", "<path to spark-{{site.SPARK_VERSION}}.tar.gz uploaded above>")
+val sc = new SparkContext(conf)
{% 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.
-
# Mesos Run Modes
Spark can run over Mesos in two modes: "fine-grained" and "coarse-grained". In fine-grained mode, which is the default,
@@ -34,17 +35,15 @@ launch only *one* long-running Spark task on each Mesos machine, and dynamically
it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration
of the application.
-To run in coarse-grained mode, set the `spark.mesos.coarse` system property to true *before* creating your SparkContext:
+To run in coarse-grained mode, set the `spark.mesos.coarse` property in your [SparkConf](configuration.html#spark-properties):
{% highlight scala %}
-System.setProperty("spark.mesos.coarse", "true")
-val sc = new SparkContext("mesos://HOST:5050", "App Name", ...)
+conf.set("spark.mesos.coarse", "true")
{% 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 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.
+application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example).
# Running Alongside Hadoop
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 4056e9c15d..b206270107 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -12,11 +12,12 @@ was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0.
We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster.
This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows:
- SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly
+ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly
The assembled JAR will be something like this:
`./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`.
+The build process now also supports new YARN versions (2.2.x). See below.
# Preparations
@@ -31,27 +32,34 @@ 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.
-* 'spark.yarn.scheduler.heartbeat.interval-ms', the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds.
-* 'spark.yarn.max.worker.failures', the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3.
+
+* `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.
+* `spark.yarn.scheduler.heartbeat.interval-ms`, the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds.
+* `spark.yarn.max.worker.failures`, the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3.
# 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.
This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager.
+There are two scheduler mode that can be used to launch spark application on YARN.
+
+## Launch spark application by YARN Client with yarn-standalone mode.
+
The command to launch the YARN Client is as follows:
- SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./spark-class org.apache.spark.deploy.yarn.Client \
+ SPARK_JAR=<SPARK_ASSEMBLY_JAR_FILE> ./bin/spark-class org.apache.spark.deploy.yarn.Client \
--jar <YOUR_APP_JAR_FILE> \
--class <APP_MAIN_CLASS> \
--args <APP_MAIN_ARGUMENTS> \
--num-workers <NUMBER_OF_WORKER_MACHINES> \
+ --master-class <ApplicationMaster_CLASS>
--master-memory <MEMORY_FOR_MASTER> \
--worker-memory <MEMORY_PER_WORKER> \
--worker-cores <CORES_PER_WORKER> \
@@ -64,14 +72,14 @@ The command to launch the YARN Client is as follows:
For example:
# Build the Spark assembly JAR and the Spark examples JAR
- $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true ./sbt/sbt assembly
+ $ 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 \
+ ./bin/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 \
@@ -85,11 +93,35 @@ For example:
$ 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.
+The above starts a YARN Client programs which start the default Application Master. Then SparkPi will be run as a child thread of Application Master, YARN Client will periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running.
+
+With this mode, your application is actually run on the remote machine where the Application Master is run upon. Thus application that involve local interaction will not work well, e.g. spark-shell.
+
+## Launch spark application with yarn-client mode.
+
+With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR
+
+In order to tune worker core/number/memory etc. You need to export SPARK_WORKER_CORES, SPARK_WORKER_MEMORY, SPARK_WORKER_INSTANCES e.g. by ./conf/spark-env.sh
+
+For example:
+
+ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
+ SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
+ ./bin/run-example org.apache.spark.examples.SparkPi yarn-client
+
+
+ SPARK_JAR=./assembly/target/scala-{{site.SCALA_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \
+ SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
+ MASTER=yarn-client ./bin/spark-shell
+
+You can also send extra files to yarn cluster for worker to use by exporting SPARK_YARN_DIST_FILES=file1,file2... etc.
+
+# Building Spark for Hadoop/YARN 2.2.x
+
+See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process.
# Important Notes
-- 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.
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 94e8563a8b..c1ef46a1cd 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -13,7 +13,7 @@ At a high level, every Spark application consists of a *driver program* that run
A second abstraction in Spark is *shared variables* that can be used in parallel operations. By default, when Spark runs a function in parallel as a set of tasks on different nodes, it ships a copy of each variable used in the function to each task. Sometimes, a variable needs to be shared across tasks, or between tasks and the driver program. Spark supports two types of shared variables: *broadcast variables*, which can be used to cache a value in memory on all nodes, and *accumulators*, which are variables that are only "added" to, such as counters and sums.
-This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `spark-shell` script. We highly recommend doing that to follow along!
+This guide shows each of these features and walks through some samples. It assumes some familiarity with Scala, especially with the syntax for [closures](http://www.scala-lang.org/node/133). Note that you can also run Spark interactively using the `bin/spark-shell` script. We highly recommend doing that to follow along!
# Linking with Spark
@@ -49,18 +49,21 @@ This is done through the following constructor:
new SparkContext(master, appName, [sparkHome], [jars])
{% endhighlight %}
+or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+object for more advanced configuration.
+
The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later.
-In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `spark-shell` on four cores, use
+In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on four cores, use
{% highlight bash %}
-$ MASTER=local[4] ./spark-shell
+$ MASTER=local[4] ./bin/spark-shell
{% endhighlight %}
Or, to also add `code.jar` to its classpath, use:
{% highlight bash %}
-$ MASTER=local[4] ADD_JARS=code.jar ./spark-shell
+$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell
{% endhighlight %}
### Master URLs
@@ -92,8 +95,7 @@ If you want to run your application on a cluster, you will need to specify the t
* `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 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.
-
+If you run `bin/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 ./bin/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.
# Resilient Distributed Datasets (RDDs)
@@ -363,10 +365,10 @@ res2: Int = 10
# Where to Go from Here
-You can see some [example Spark programs](http://www.spark-project.org/examples.html) on the Spark website.
-In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `run-example` script included in Spark; for example:
+You can see some [example Spark programs](http://spark.incubator.apache.org/examples.html) on the Spark website.
+In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example:
- ./run-example org.apache.spark.examples.SparkPi
+ ./bin/run-example org.apache.spark.examples.SparkPi
Each example program prints usage help when run without any arguments.
diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md
index d6315d97f4..11c51d5cde 100644
--- a/docs/spark-debugger.md
+++ b/docs/spark-debugger.md
@@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP
### Loading the event log into the debugger
-1. Run a Spark shell with `MASTER=<i>host</i> ./spark-shell`.
+1. Run a Spark shell with `MASTER=<i>host</i> ./bin/spark-shell`.
2. Use `EventLogReader` to load the event log as follows:
{% highlight scala %}
spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log"))
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 17066ef0dd..f47d41f966 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -20,7 +20,7 @@ then modify `conf/spark-env.sh` in the `dist/` directory before deploying to all
You can start a standalone master server by executing:
- ./bin/start-master.sh
+ ./sbin/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
@@ -28,7 +28,7 @@ the master's web UI, which is [http://localhost:8080](http://localhost:8080) by
Similarly, you can start one or more workers and connect them to the master via:
- ./spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
+ ./bin/spark-class org.apache.spark.deploy.worker.Worker spark://IP:PORT
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).
@@ -51,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 applicatons to use on the machine (default: all available); only on worker</td>
+ <td>Total CPU cores to allow Spark applications 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 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>
+ <td>Total amount of memory to allow Spark applications 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>
@@ -70,12 +70,12 @@ To launch a Spark standalone cluster with the launch scripts, you need to create
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`:
-- `bin/start-master.sh` - Starts a master instance on the machine the script is executed on.
-- `bin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
-- `bin/start-all.sh` - Starts both a master and a number of slaves as described above.
-- `bin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
-- `bin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
-- `bin/stop-all.sh` - Stops both the master and the slaves as described above.
+- `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on.
+- `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file.
+- `sbin/start-all.sh` - Starts both a master and a number of slaves as described above.
+- `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script.
+- `sbin/stop-slaves.sh` - Stops the slave instances that were started via `bin/start-slaves.sh`.
+- `sbin/stop-all.sh` - Stops both the master and the slaves as described above.
Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine.
@@ -143,9 +143,9 @@ 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
+ MASTER=spark://IP:PORT ./bin/spark-shell
-Note that if you are running spark-shell from one of the spark cluster machines, the `spark-shell` script will
+Note that if you are running spark-shell from one of the spark cluster machines, the `bin/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.
@@ -154,12 +154,29 @@ You can also pass an option `-c <numCores>` to control the number of cores that
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.
+application will use.
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.
-
+application at a time. You can cap the number of cores by setting `spark.cores.max` in your
+[SparkConf](configuration.html#spark-properties). For example:
+
+{% highlight scala %}
+val conf = new SparkConf()
+ .setMaster(...)
+ .setAppName(...)
+ .set("spark.cores.max", "10")
+val sc = new SparkContext(conf)
+{% endhighlight %}
+
+In addition, you can configure `spark.deploy.defaultCores` on the cluster master process to change the
+default for applications that don't set `spark.cores.max` to something less than infinite.
+Do this by adding the following to `conf/spark-env.sh`:
+
+{% highlight bash %}
+export SPARK_JAVA_OPTS="-Dspark.deploy.defaultCores=<value>"
+{% endhighlight %}
+
+This is useful on shared clusters where users might not have configured a maximum number of cores
+individually.
# Monitoring and Logging
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 851e30fe76..1c9ece6270 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -214,7 +214,7 @@ ssc.stop()
{% endhighlight %}
# Example
-A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/NetworkWordCount.scala` .
+A simple example to start off is the [NetworkWordCount](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala). This example counts the words received from a network server every second. Given below is the relevant sections of the source code. You can find the full source code in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala` .
{% highlight scala %}
import org.apache.spark.streaming.{Seconds, StreamingContext}
@@ -245,7 +245,7 @@ $ nc -lk 9999
Then, in a different terminal, you can start NetworkWordCount by using
{% highlight bash %}
-$ ./run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
{% endhighlight %}
This will make NetworkWordCount connect to the netcat server. Any lines typed in the terminal running the netcat server will be counted and printed on screen.
@@ -283,7 +283,7 @@ Time: 1357008430000 ms
</td>
</table>
-You can find more examples in `<Spark repo>/streaming/src/main/scala/spark/streaming/examples/`. They can be run in the similar manner using `./run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
+You can find more examples in `<Spark repo>/streaming/src/main/scala/org/apache/spark/streaming/examples/`. They can be run in the similar manner using `./bin/run-example org.apache.spark.streaming.examples....` . Executing without any parameter would give the required parameter list. Further explanation to run them can be found in comments in the files.
# DStream Persistence
Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. Hence, DStreams generated by window-based operations are automatically persisted in memory, without the developer calling `persist()`.
@@ -326,7 +326,7 @@ Getting the best performance of a Spark Streaming application on a cluster requi
There are a number of optimizations that can be done in Spark to minimize the processing time of each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones.
### Level of Parallelism
-Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the system property `spark.default.parallelism` to change the default.
+Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the [`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default.
### Data Serialization
The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it.
@@ -349,7 +349,7 @@ For a Spark Streaming application running on a cluster to be stable, the process
A good approach to figure out the right batch size for your application is to test it with a conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (in the Spark master logs, find the line having the phrase "Total delay"). If the delay is maintained to be less than the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the data rate and/or reducing the batch size. Note that momentary increase in the delay due to temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size).
## 24/7 Operation
-By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the Java system property `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
+By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic cleanup of it metadata. This can be enabled by setting the [config property](configuration.html#spark-properties) `spark.cleaner.ttl` to the number of seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. Note, that this property needs to be set before the SparkContext is created.
This value is closely tied with any window operation that is being used. Any window operation would require the input data to be persisted in memory for at least the duration of the window. Hence it is necessary to set the delay to at least the value of the largest window operation used in the Spark Streaming application. If this delay is set too low, the application will throw an exception saying so.
@@ -483,7 +483,7 @@ Similar to [Spark's Java API](java-programming-guide.html), we also provide a Ja
1. Functions for transformations must be implemented as subclasses of [Function](api/core/index.html#org.apache.spark.api.java.function.Function) and [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2)
1. Unlike the Scala API, the Java API handles DStreams for key-value pairs using a separate [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream) class(similar to [JavaRDD and JavaPairRDD](java-programming-guide.html#rdd-classes). DStream functions like `map` and `filter` are implemented separately by JavaDStreams and JavaPairDStream to return DStreams of appropriate types.
-Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `<spark repo>/examples/src/main/java/spark/streaming/examples/JavaNetworkWordCount.java`
+Spark's [Java Programming Guide](java-programming-guide.html) gives more ideas about using the Java API. To extends the ideas presented for the RDDs to DStreams, we present parts of the Java version of the same NetworkWordCount example presented above. The full source code is given at `<spark repo>/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java`
The streaming context and the socket stream from input source is started by using a `JavaStreamingContext`, that has the same parameters and provides the same input streams as its Scala counterpart.
@@ -527,5 +527,5 @@ JavaPairDStream<String, Integer> wordCounts = words.map(
# Where to Go from Here
* API docs - [Scala](api/streaming/index.html#org.apache.spark.streaming.package) and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package)
-* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/spark/streaming/examples)
+* More examples - [Scala](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/streaming/examples)
* [Paper describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf)
diff --git a/docs/tuning.md b/docs/tuning.md
index f33fda37eb..bbb870085c 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -38,13 +38,15 @@ in your operations) and performance. It provides two serialization libraries:
`Serializable` types and requires you to *register* the classes you'll use in the program in advance
for best performance.
-You can switch to using Kryo by calling `System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`
-*before* creating your SparkContext. The only reason it is not the default is because of the custom
+You can switch to using Kryo by initializing your job with a [SparkConf](configuration.html#spark-properties)
+and calling `conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")`.
+This setting configures the serializer used for not only shuffling data between worker
+nodes but also when serializing RDDs to disk. The only reason Kryo is not the default is because of the custom
registration requirement, but we recommend trying it in any network-intensive application.
Finally, to register your classes with Kryo, create a public class that extends
[`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the
-`spark.kryo.registrator` system property to point to it, as follows:
+`spark.kryo.registrator` config property to point to it, as follows:
{% highlight scala %}
import com.esotericsoftware.kryo.Kryo
@@ -57,17 +59,17 @@ class MyRegistrator extends KryoRegistrator {
}
}
-// Make sure to set these properties *before* creating a SparkContext!
-System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
-System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator")
-val sc = new SparkContext(...)
+val conf = new SparkConf().setMaster(...).setAppName(...)
+conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+conf.set("spark.kryo.registrator", "mypackage.MyRegistrator")
+val sc = new SparkContext(conf)
{% endhighlight %}
The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced
registration options, such as adding custom serialization code.
If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb`
-system property. The default is 2, but this value needs to be large enough to hold the *largest*
+config property. The default is 2, but this value needs to be large enough to hold the *largest*
object you will serialize.
Finally, if you don't register your classes, Kryo will still work, but it will have to store the
@@ -164,7 +166,7 @@ cache RDDs. This means that 33% of memory is available for any objects created d
In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of
memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call
-`System.setProperty("spark.storage.memoryFraction", "0.5")`. Combined with the use of serialized caching,
+`conf.set("spark.storage.memoryFraction", "0.5")` on your SparkConf. Combined with the use of serialized caching,
using a smaller cache should be sufficient to mitigate most of the garbage collection problems.
In case you are interested in further tuning the Java GC, continue reading below.
@@ -218,7 +220,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a
distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest
parent RDD's number of partitions. You can pass the level of parallelism as a second argument
(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation),
-or set the system property `spark.default.parallelism` to change the default.
+or set the config property `spark.default.parallelism` to change the default.
In general, we recommend 2-3 tasks per CPU core in your cluster.
## Memory Usage of Reduce Tasks
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 1189232428..d82a1e1490 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -436,7 +436,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key):
def setup_standalone_cluster(master, slave_nodes, opts):
slave_ips = '\n'.join([i.public_dns_name for i in slave_nodes])
ssh(master, opts, "echo \"%s\" > spark/conf/slaves" % (slave_ips))
- ssh(master, opts, "/root/spark/bin/start-all.sh")
+ ssh(master, opts, "/root/spark/sbin/start-all.sh")
def setup_spark_cluster(master, opts):
ssh(master, opts, "chmod u+x spark-ec2/setup.sh")
@@ -589,7 +589,7 @@ def ssh(host, opts, command):
while True:
try:
return subprocess.check_call(
- ssh_command(opts) + ['-t', '%s@%s' % (opts.user, host), stringify_command(command)])
+ ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), stringify_command(command)])
except subprocess.CalledProcessError as e:
if (tries > 2):
# If this was an ssh failure, provide the user with hints.
@@ -730,7 +730,7 @@ def real_main():
if opts.proxy_port != None:
proxy_opt = ['-D', opts.proxy_port]
subprocess.check_call(
- ssh_command(opts) + proxy_opt + ['-t', "%s@%s" % (opts.user, master)])
+ ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)])
elif action == "get-master":
(master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name)
diff --git a/examples/pom.xml b/examples/pom.xml
index aee371fbc7..cb4f7ee33b 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-examples_2.9.3</artifactId>
+ <artifactId>spark-examples_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Examples</name>
<url>http://spark.incubator.apache.org/</url>
@@ -49,29 +49,54 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-twitter_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-kafka_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-flume_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-zeromq_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-mqtt_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase</artifactId>
<version>0.94.6</version>
@@ -87,43 +112,28 @@
</exclusions>
</dependency>
<dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka_2.9.2</artifactId>
- <version>0.8.0-beta1</version>
- <exclusions>
- <exclusion>
- <groupId>com.sun.jmx</groupId>
- <artifactId>jmxri</artifactId>
- </exclusion>
- <exclusion>
- <groupId>com.sun.jdmk</groupId>
- <artifactId>jmxtools</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
<groupId>org.eclipse.jetty</groupId>
<artifactId>jetty-server</artifactId>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>algebird-core_2.9.2</artifactId>
+ <artifactId>algebird-core_${scala.binary.version}</artifactId>
<version>0.1.11</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.cassandra</groupId>
<artifactId>cassandra-all</artifactId>
- <version>1.2.5</version>
+ <version>1.2.6</version>
<exclusions>
<exclusion>
<groupId>com.google.guava</groupId>
@@ -166,15 +176,15 @@
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-shade-plugin</artifactId>
<configuration>
<shadedArtifactAttached>false</shadedArtifactAttached>
- <outputFile>${project.build.directory}/scala-${scala.version}/${project.artifactId}-assembly-${project.version}.jar</outputFile>
+ <outputFile>${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar</outputFile>
<artifactSet>
<includes>
<include>*:*</include>
@@ -203,6 +213,9 @@
<transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
<resource>reference.conf</resource>
</transformer>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
+ <resource>log4j.properties</resource>
+ </transformer>
</transformers>
</configuration>
</execution>
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
index be0d38589c..d552c47b22 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java
@@ -24,19 +24,19 @@ import org.apache.spark.api.java.function.Function2;
import java.io.Serializable;
import java.util.Arrays;
-import java.util.StringTokenizer;
import java.util.Random;
+import java.util.regex.Pattern;
/**
* Logistic regression based classification.
*/
-public class JavaHdfsLR {
+public final class JavaHdfsLR {
- static int D = 10; // Number of dimensions
- static Random rand = new Random(42);
+ private static final int D = 10; // Number of dimensions
+ private static final Random rand = new Random(42);
static class DataPoint implements Serializable {
- public DataPoint(double[] x, double y) {
+ DataPoint(double[] x, double y) {
this.x = x;
this.y = y;
}
@@ -46,20 +46,22 @@ public class JavaHdfsLR {
}
static class ParsePoint extends Function<String, DataPoint> {
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ @Override
public DataPoint call(String line) {
- StringTokenizer tok = new StringTokenizer(line, " ");
- double y = Double.parseDouble(tok.nextToken());
+ String[] tok = SPACE.split(line);
+ double y = Double.parseDouble(tok[0]);
double[] x = new double[D];
- int i = 0;
- while (i < D) {
- x[i] = Double.parseDouble(tok.nextToken());
- i += 1;
+ for (int i = 0; i < D; i++) {
+ x[i] = Double.parseDouble(tok[i + 1]);
}
return new DataPoint(x, y);
}
}
static class VectorSum extends Function2<double[], double[], double[]> {
+ @Override
public double[] call(double[] a, double[] b) {
double[] result = new double[D];
for (int j = 0; j < D; j++) {
@@ -70,12 +72,13 @@ public class JavaHdfsLR {
}
static class ComputeGradient extends Function<DataPoint, double[]> {
- double[] weights;
+ private final double[] weights;
- public ComputeGradient(double[] weights) {
+ ComputeGradient(double[] weights) {
this.weights = weights;
}
+ @Override
public double[] call(DataPoint p) {
double[] gradient = new double[D];
for (int i = 0; i < D; i++) {
@@ -106,7 +109,7 @@ public class JavaHdfsLR {
}
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaHdfsLR",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaHdfsLR.class));
JavaRDD<String> lines = sc.textFile(args[1]);
JavaRDD<DataPoint> points = lines.map(new ParsePoint()).cache();
int ITERATIONS = Integer.parseInt(args[2]);
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
index 5a6afe7eae..0dc879275a 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java
@@ -27,19 +27,24 @@ import org.apache.spark.util.Vector;
import java.util.List;
import java.util.Map;
+import java.util.regex.Pattern;
/**
* K-means clustering using Java API.
*/
-public class JavaKMeans {
+public final class JavaKMeans {
+
+ private static final Pattern SPACE = Pattern.compile(" ");
/** Parses numbers split by whitespace to a vector */
static Vector parseVector(String line) {
- String[] splits = line.split(" ");
+ String[] splits = SPACE.split(line);
double[] data = new double[splits.length];
int i = 0;
- for (String s : splits)
- data[i] = Double.parseDouble(splits[i++]);
+ for (String s : splits) {
+ data[i] = Double.parseDouble(s);
+ i++;
+ }
return new Vector(data);
}
@@ -74,7 +79,7 @@ public class JavaKMeans {
System.exit(1);
}
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class));
String path = args[1];
int K = Integer.parseInt(args[2]);
double convergeDist = Double.parseDouble(args[3]);
@@ -82,7 +87,7 @@ public class JavaKMeans {
JavaRDD<Vector> data = sc.textFile(path).map(
new Function<String, Vector>() {
@Override
- public Vector call(String line) throws Exception {
+ public Vector call(String line) {
return parseVector(line);
}
}
@@ -96,7 +101,7 @@ public class JavaKMeans {
JavaPairRDD<Integer, Vector> closest = data.map(
new PairFunction<Vector, Integer, Vector>() {
@Override
- public Tuple2<Integer, Vector> call(Vector vector) throws Exception {
+ public Tuple2<Integer, Vector> call(Vector vector) {
return new Tuple2<Integer, Vector>(
closestPoint(vector, centroids), vector);
}
@@ -107,7 +112,8 @@ public class JavaKMeans {
JavaPairRDD<Integer, List<Vector>> pointsGroup = closest.groupByKey();
Map<Integer, Vector> newCentroids = pointsGroup.mapValues(
new Function<List<Vector>, Vector>() {
- public Vector call(List<Vector> ps) throws Exception {
+ @Override
+ public Vector call(List<Vector> ps) {
return average(ps);
}
}).collectAsMap();
@@ -122,8 +128,9 @@ public class JavaKMeans {
} while (tempDist > convergeDist);
System.out.println("Final centers:");
- for (Vector c : centroids)
+ for (Vector c : centroids) {
System.out.println(c);
+ }
System.exit(0);
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
index 152f029213..9eb1cadd71 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
@@ -35,9 +35,9 @@ import java.util.regex.Pattern;
/**
* Executes a roll up-style query against Apache logs.
*/
-public class JavaLogQuery {
+public final class JavaLogQuery {
- public static List<String> exampleApacheLogs = Lists.newArrayList(
+ public static final List<String> exampleApacheLogs = Lists.newArrayList(
"10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " +
"HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " +
"Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " +
@@ -51,14 +51,14 @@ public class JavaLogQuery {
"3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " +
"0 73.23.2.15 images.com 1358492557 - Whatup");
- public static Pattern apacheLogRegex = Pattern.compile(
+ public static final Pattern apacheLogRegex = Pattern.compile(
"^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*");
/** Tracks the total query count and number of aggregate bytes for a particular group. */
public static class Stats implements Serializable {
- private int count;
- private int numBytes;
+ private final int count;
+ private final int numBytes;
public Stats(int count, int numBytes) {
this.count = count;
@@ -92,38 +92,38 @@ public class JavaLogQuery {
if (m.find()) {
int bytes = Integer.parseInt(m.group(7));
return new Stats(1, bytes);
- }
- else
+ } else {
return new Stats(1, 0);
+ }
}
- public static void main(String[] args) throws Exception {
+ public static void main(String[] args) {
if (args.length == 0) {
System.err.println("Usage: JavaLogQuery <master> [logFile]");
System.exit(1);
}
JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLogQuery.class));
JavaRDD<String> dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs);
JavaPairRDD<Tuple3<String, String, String>, Stats> extracted = dataSet.map(new PairFunction<String, Tuple3<String, String, String>, Stats>() {
@Override
- public Tuple2<Tuple3<String, String, String>, Stats> call(String s) throws Exception {
+ public Tuple2<Tuple3<String, String, String>, Stats> call(String s) {
return new Tuple2<Tuple3<String, String, String>, Stats>(extractKey(s), extractStats(s));
}
});
JavaPairRDD<Tuple3<String, String, String>, Stats> counts = extracted.reduceByKey(new Function2<Stats, Stats, Stats>() {
@Override
- public Stats call(Stats stats, Stats stats2) throws Exception {
+ public Stats call(Stats stats, Stats stats2) {
return stats.merge(stats2);
}
});
List<Tuple2<Tuple3<String, String, String>, Stats>> output = counts.collect();
- for (Tuple2 t : output) {
+ for (Tuple2<?,?> t : output) {
System.out.println(t._1 + "\t" + t._2);
}
System.exit(0);
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
index c5603a639b..a84245b0c7 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java
@@ -21,7 +21,6 @@ import scala.Tuple2;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFlatMapFunction;
@@ -29,6 +28,7 @@ import org.apache.spark.api.java.function.PairFunction;
import java.util.List;
import java.util.ArrayList;
+import java.util.regex.Pattern;
/**
* Computes the PageRank of URLs from an input file. Input file should
@@ -39,7 +39,9 @@ import java.util.ArrayList;
* ...
* where URL and their neighbors are separated by space(s).
*/
-public class JavaPageRank {
+public final class JavaPageRank {
+ private static final Pattern SPACES = Pattern.compile("\\s+");
+
private static class Sum extends Function2<Double, Double, Double> {
@Override
public Double call(Double a, Double b) {
@@ -54,7 +56,7 @@ public class JavaPageRank {
}
JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaPageRank",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaPageRank.class));
// Loads in input file. It should be in format of:
// URL neighbor URL
@@ -67,7 +69,7 @@ public class JavaPageRank {
JavaPairRDD<String, List<String>> links = lines.map(new PairFunction<String, String, String>() {
@Override
public Tuple2<String, String> call(String s) {
- String[] parts = s.split("\\s+");
+ String[] parts = SPACES.split(s);
return new Tuple2<String, String>(parts[0], parts[1]);
}
}).distinct().groupByKey().cache();
@@ -75,7 +77,7 @@ public class JavaPageRank {
// Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one.
JavaPairRDD<String, Double> ranks = links.mapValues(new Function<List<String>, Double>() {
@Override
- public Double call(List<String> rs) throws Exception {
+ public Double call(List<String> rs) {
return 1.0;
}
});
@@ -98,7 +100,7 @@ public class JavaPageRank {
// Re-calculates URL ranks based on neighbor contributions.
ranks = contribs.reduceByKey(new Sum()).mapValues(new Function<Double, Double>() {
@Override
- public Double call(Double sum) throws Exception {
+ public Double call(Double sum) {
return 0.15 + sum * 0.85;
}
});
@@ -106,7 +108,7 @@ public class JavaPageRank {
// Collects all URL ranks and dump them to console.
List<Tuple2<String, Double>> output = ranks.collect();
- for (Tuple2 tuple : output) {
+ for (Tuple2<?,?> tuple : output) {
System.out.println(tuple._1 + " has rank: " + tuple._2 + ".");
}
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
index 4a2380caf5..3ec4a58d48 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java
@@ -26,8 +26,7 @@ import java.util.ArrayList;
import java.util.List;
/** Computes an approximation to pi */
-public class JavaSparkPi {
-
+public final class JavaSparkPi {
public static void main(String[] args) throws Exception {
if (args.length == 0) {
@@ -36,26 +35,27 @@ public class JavaSparkPi {
}
JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkPi.class));
int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2;
int n = 100000 * slices;
List<Integer> l = new ArrayList<Integer>(n);
- for (int i = 0; i < n; i++)
+ for (int i = 0; i < n; i++) {
l.add(i);
+ }
JavaRDD<Integer> dataSet = jsc.parallelize(l, slices);
int count = dataSet.map(new Function<Integer, Integer>() {
@Override
- public Integer call(Integer integer) throws Exception {
+ public Integer call(Integer integer) {
double x = Math.random() * 2 - 1;
double y = Math.random() * 2 - 1;
return (x * x + y * y < 1) ? 1 : 0;
}
}).reduce(new Function2<Integer, Integer, Integer>() {
@Override
- public Integer call(Integer integer, Integer integer2) throws Exception {
+ public Integer call(Integer integer, Integer integer2) {
return integer + integer2;
}
});
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
index 17f21f6b77..2ceb0fd94b 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
@@ -31,11 +31,11 @@ import java.util.Set;
/**
* Transitive closure on a graph, implemented in Java.
*/
-public class JavaTC {
+public final class JavaTC {
- static int numEdges = 200;
- static int numVertices = 100;
- static Random rand = new Random(42);
+ private static final int numEdges = 200;
+ private static final int numVertices = 100;
+ private static final Random rand = new Random(42);
static List<Tuple2<Integer, Integer>> generateGraph() {
Set<Tuple2<Integer, Integer>> edges = new HashSet<Tuple2<Integer, Integer>>(numEdges);
@@ -43,15 +43,18 @@ public class JavaTC {
int from = rand.nextInt(numVertices);
int to = rand.nextInt(numVertices);
Tuple2<Integer, Integer> e = new Tuple2<Integer, Integer>(from, to);
- if (from != to) edges.add(e);
+ if (from != to) {
+ edges.add(e);
+ }
}
return new ArrayList<Tuple2<Integer, Integer>>(edges);
}
static class ProjectFn extends PairFunction<Tuple2<Integer, Tuple2<Integer, Integer>>,
Integer, Integer> {
- static ProjectFn INSTANCE = new ProjectFn();
+ static final ProjectFn INSTANCE = new ProjectFn();
+ @Override
public Tuple2<Integer, Integer> call(Tuple2<Integer, Tuple2<Integer, Integer>> triple) {
return new Tuple2<Integer, Integer>(triple._2()._2(), triple._2()._1());
}
@@ -64,7 +67,7 @@ public class JavaTC {
}
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaTC",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaTC.class));
Integer slices = (args.length > 1) ? Integer.parseInt(args[1]): 2;
JavaPairRDD<Integer, Integer> tc = sc.parallelizePairs(generateGraph(), slices).cache();
@@ -76,6 +79,7 @@ public class JavaTC {
// Because join() joins on keys, the edges are stored in reversed order.
JavaPairRDD<Integer, Integer> edges = tc.map(
new PairFunction<Tuple2<Integer, Integer>, Integer, Integer>() {
+ @Override
public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> e) {
return new Tuple2<Integer, Integer>(e._2(), e._1());
}
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
index 07d32ad659..6651f98d56 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java
@@ -27,8 +27,11 @@ import org.apache.spark.api.java.function.PairFunction;
import java.util.Arrays;
import java.util.List;
+import java.util.regex.Pattern;
+
+public final class JavaWordCount {
+ private static final Pattern SPACE = Pattern.compile(" ");
-public class JavaWordCount {
public static void main(String[] args) throws Exception {
if (args.length < 2) {
System.err.println("Usage: JavaWordCount <master> <file>");
@@ -36,29 +39,32 @@ public class JavaWordCount {
}
JavaSparkContext ctx = new JavaSparkContext(args[0], "JavaWordCount",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaWordCount.class));
JavaRDD<String> lines = ctx.textFile(args[1], 1);
JavaRDD<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
+ @Override
public Iterable<String> call(String s) {
- return Arrays.asList(s.split(" "));
+ return Arrays.asList(SPACE.split(s));
}
});
JavaPairRDD<String, Integer> ones = words.map(new PairFunction<String, String, Integer>() {
+ @Override
public Tuple2<String, Integer> call(String s) {
return new Tuple2<String, Integer>(s, 1);
}
});
JavaPairRDD<String, Integer> counts = ones.reduceByKey(new Function2<Integer, Integer, Integer>() {
+ @Override
public Integer call(Integer i1, Integer i2) {
return i1 + i2;
}
});
List<Tuple2<String, Integer>> output = counts.collect();
- for (Tuple2 tuple : output) {
+ for (Tuple2<?,?> tuple : output) {
System.out.println(tuple._1 + ": " + tuple._2);
}
System.exit(0);
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
index 628cb892b6..435a86e62a 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
@@ -25,30 +25,33 @@ import org.apache.spark.mllib.recommendation.ALS;
import org.apache.spark.mllib.recommendation.MatrixFactorizationModel;
import org.apache.spark.mllib.recommendation.Rating;
-import java.io.Serializable;
import java.util.Arrays;
-import java.util.StringTokenizer;
+import java.util.regex.Pattern;
import scala.Tuple2;
/**
* Example using MLLib ALS from Java.
*/
-public class JavaALS {
+public final class JavaALS {
static class ParseRating extends Function<String, Rating> {
+ private static final Pattern COMMA = Pattern.compile(",");
+
+ @Override
public Rating call(String line) {
- StringTokenizer tok = new StringTokenizer(line, ",");
- int x = Integer.parseInt(tok.nextToken());
- int y = Integer.parseInt(tok.nextToken());
- double rating = Double.parseDouble(tok.nextToken());
+ String[] tok = COMMA.split(line);
+ int x = Integer.parseInt(tok[0]);
+ int y = Integer.parseInt(tok[1]);
+ double rating = Double.parseDouble(tok[2]);
return new Rating(x, y, rating);
}
}
static class FeaturesToString extends Function<Tuple2<Object, double[]>, String> {
+ @Override
public String call(Tuple2<Object, double[]> element) {
- return element._1().toString() + "," + Arrays.toString(element._2());
+ return element._1() + "," + Arrays.toString(element._2());
}
}
@@ -69,7 +72,7 @@ public class JavaALS {
}
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaALS",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaALS.class));
JavaRDD<String> lines = sc.textFile(args[1]);
JavaRDD<Rating> ratings = lines.map(new ParseRating());
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
index cd59a139b9..4b2658f257 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
@@ -25,20 +25,22 @@ import org.apache.spark.mllib.clustering.KMeans;
import org.apache.spark.mllib.clustering.KMeansModel;
import java.util.Arrays;
-import java.util.StringTokenizer;
+import java.util.regex.Pattern;
/**
* Example using MLLib KMeans from Java.
*/
-public class JavaKMeans {
+public final class JavaKMeans {
static class ParsePoint extends Function<String, double[]> {
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ @Override
public double[] call(String line) {
- StringTokenizer tok = new StringTokenizer(line, " ");
- int numTokens = tok.countTokens();
- double[] point = new double[numTokens];
- for (int i = 0; i < numTokens; ++i) {
- point[i] = Double.parseDouble(tok.nextToken());
+ String[] tok = SPACE.split(line);
+ double[] point = new double[tok.length];
+ for (int i = 0; i < tok.length; ++i) {
+ point[i] = Double.parseDouble(tok[i]);
}
return point;
}
@@ -62,7 +64,7 @@ public class JavaKMeans {
}
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class));
JavaRDD<String> lines = sc.textFile(args[1]);
JavaRDD<double[]> points = lines.map(new ParsePoint());
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
index 258061c8e6..21586ce817 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
@@ -27,22 +27,25 @@ import org.apache.spark.mllib.classification.LogisticRegressionModel;
import org.apache.spark.mllib.regression.LabeledPoint;
import java.util.Arrays;
-import java.util.StringTokenizer;
+import java.util.regex.Pattern;
/**
* Logistic regression based classification using ML Lib.
*/
-public class JavaLR {
+public final class JavaLR {
static class ParsePoint extends Function<String, LabeledPoint> {
+ private static final Pattern COMMA = Pattern.compile(",");
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ @Override
public LabeledPoint call(String line) {
- String[] parts = line.split(",");
+ String[] parts = COMMA.split(line);
double y = Double.parseDouble(parts[0]);
- StringTokenizer tok = new StringTokenizer(parts[1], " ");
- int numTokens = tok.countTokens();
- double[] x = new double[numTokens];
- for (int i = 0; i < numTokens; ++i) {
- x[i] = Double.parseDouble(tok.nextToken());
+ String[] tok = SPACE.split(parts[1]);
+ double[] x = new double[tok.length];
+ for (int i = 0; i < tok.length; ++i) {
+ x[i] = Double.parseDouble(tok[i]);
}
return new LabeledPoint(y, x);
}
@@ -59,7 +62,7 @@ public class JavaLR {
}
JavaSparkContext sc = new JavaSparkContext(args[0], "JavaLR",
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaLR.class));
JavaRDD<String> lines = sc.textFile(args[1]);
JavaRDD<LabeledPoint> points = lines.map(new ParsePoint()).cache();
double stepSize = Double.parseDouble(args[2]);
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
index 261813bf2f..b11cfa667e 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
@@ -20,7 +20,8 @@ package org.apache.spark.streaming.examples;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.streaming.*;
import org.apache.spark.streaming.api.java.*;
-import org.apache.spark.streaming.dstream.SparkFlumeEvent;
+import org.apache.spark.streaming.flume.FlumeUtils;
+import org.apache.spark.streaming.flume.SparkFlumeEvent;
/**
* Produces a count of events received from Flume.
@@ -36,7 +37,10 @@ import org.apache.spark.streaming.dstream.SparkFlumeEvent;
* creates a server and listens for flume events.
* <port> is the port the Flume receiver will listen on.
*/
-public class JavaFlumeEventCount {
+public final class JavaFlumeEventCount {
+ private JavaFlumeEventCount() {
+ }
+
public static void main(String[] args) {
if (args.length != 3) {
System.err.println("Usage: JavaFlumeEventCount <master> <host> <port>");
@@ -49,10 +53,10 @@ public class JavaFlumeEventCount {
Duration batchInterval = new Duration(2000);
- JavaStreamingContext sc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
-
- JavaDStream<SparkFlumeEvent> flumeStream = sc.flumeStream("localhost", port);
+ JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
+ System.getenv("SPARK_HOME"),
+ JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class));
+ JavaDStream<SparkFlumeEvent> flumeStream = FlumeUtils.createStream(ssc, "localhost", port);
flumeStream.count();
@@ -63,6 +67,6 @@ public class JavaFlumeEventCount {
}
}).print();
- sc.start();
+ ssc.start();
}
}
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
index 9a8e4209ed..16b8a948e6 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
@@ -19,6 +19,7 @@ package org.apache.spark.streaming.examples;
import java.util.Map;
import java.util.HashMap;
+import java.util.regex.Pattern;
import com.google.common.collect.Lists;
import org.apache.spark.api.java.function.FlatMapFunction;
@@ -29,6 +30,7 @@ import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.apache.spark.streaming.kafka.KafkaUtils;
import scala.Tuple2;
/**
@@ -41,11 +43,16 @@ import scala.Tuple2;
* <numThreads> is the number of threads the kafka consumer should use
*
* Example:
- * `./run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
+ * `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
* zoo03 my-consumer-group topic1,topic2 1`
*/
-public class JavaKafkaWordCount {
+public final class JavaKafkaWordCount {
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ private JavaKafkaWordCount() {
+ }
+
public static void main(String[] args) {
if (args.length < 5) {
System.err.println("Usage: KafkaWordCount <master> <zkQuorum> <group> <topics> <numThreads>");
@@ -53,8 +60,9 @@ public class JavaKafkaWordCount {
}
// Create the context with a 1 second batch size
- JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount",
- new Duration(2000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ JavaStreamingContext jssc = new JavaStreamingContext(args[0], "KafkaWordCount",
+ new Duration(2000), System.getenv("SPARK_HOME"),
+ JavaStreamingContext.jarOfClass(JavaKafkaWordCount.class));
int numThreads = Integer.parseInt(args[4]);
Map<String, Integer> topicMap = new HashMap<String, Integer>();
@@ -63,11 +71,11 @@ public class JavaKafkaWordCount {
topicMap.put(topic, numThreads);
}
- JavaPairDStream<String, String> messages = ssc.kafkaStream(args[1], args[2], topicMap);
+ JavaPairDStream<String, String> messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap);
JavaDStream<String> lines = messages.map(new Function<Tuple2<String, String>, String>() {
@Override
- public String call(Tuple2<String, String> tuple2) throws Exception {
+ public String call(Tuple2<String, String> tuple2) {
return tuple2._2();
}
});
@@ -75,24 +83,24 @@ public class JavaKafkaWordCount {
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
- return Lists.newArrayList(x.split(" "));
+ return Lists.newArrayList(SPACE.split(x));
}
});
JavaPairDStream<String, Integer> wordCounts = words.map(
new PairFunction<String, String, Integer>() {
@Override
- public Tuple2<String, Integer> call(String s) throws Exception {
+ public Tuple2<String, Integer> call(String s) {
return new Tuple2<String, Integer>(s, 1);
}
}).reduceByKey(new Function2<Integer, Integer, Integer>() {
@Override
- public Integer call(Integer i1, Integer i2) throws Exception {
+ public Integer call(Integer i1, Integer i2) {
return i1 + i2;
}
});
wordCounts.print();
- ssc.start();
+ jssc.start();
}
}
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
index def87c199b..1e2efd359c 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
@@ -27,6 +27,8 @@ import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import java.util.regex.Pattern;
+
/**
* Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
* Usage: NetworkWordCount <master> <hostname> <port>
@@ -38,7 +40,12 @@ import org.apache.spark.streaming.api.java.JavaStreamingContext;
* and then run the example
* `$ ./run spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999`
*/
-public class JavaNetworkWordCount {
+public final class JavaNetworkWordCount {
+ private static final Pattern SPACE = Pattern.compile(" ");
+
+ private JavaNetworkWordCount() {
+ }
+
public static void main(String[] args) {
if (args.length < 3) {
System.err.println("Usage: NetworkWordCount <master> <hostname> <port>\n" +
@@ -48,7 +55,8 @@ public class JavaNetworkWordCount {
// Create the context with a 1 second batch size
JavaStreamingContext ssc = new JavaStreamingContext(args[0], "NetworkWordCount",
- new Duration(1000), System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ new Duration(1000), System.getenv("SPARK_HOME"),
+ JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class));
// Create a NetworkInputDStream on target ip:port and count the
// words in input stream of \n delimited test (eg. generated by 'nc')
@@ -56,18 +64,18 @@ public class JavaNetworkWordCount {
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
- return Lists.newArrayList(x.split(" "));
+ return Lists.newArrayList(SPACE.split(x));
}
});
JavaPairDStream<String, Integer> wordCounts = words.map(
new PairFunction<String, String, Integer>() {
@Override
- public Tuple2<String, Integer> call(String s) throws Exception {
+ public Tuple2<String, Integer> call(String s) {
return new Tuple2<String, Integer>(s, 1);
}
}).reduceByKey(new Function2<Integer, Integer, Integer>() {
@Override
- public Integer call(Integer i1, Integer i2) throws Exception {
+ public Integer call(Integer i1, Integer i2) {
return i1 + i2;
}
});
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
index c8c7389dd1..e05551ab83 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
+++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
@@ -31,8 +31,11 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Queue;
-public class JavaQueueStream {
- public static void main(String[] args) throws InterruptedException {
+public final class JavaQueueStream {
+ private JavaQueueStream() {
+ }
+
+ public static void main(String[] args) throws Exception {
if (args.length < 1) {
System.err.println("Usage: JavaQueueStream <master>");
System.exit(1);
@@ -40,7 +43,7 @@ public class JavaQueueStream {
// Create the context
JavaStreamingContext ssc = new JavaStreamingContext(args[0], "QueueStream", new Duration(1000),
- System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR"));
+ System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaQueueStream.class));
// Create the queue through which RDDs can be pushed to
// a QueueInputDStream
@@ -62,14 +65,14 @@ public class JavaQueueStream {
JavaPairDStream<Integer, Integer> mappedStream = inputStream.map(
new PairFunction<Integer, Integer, Integer>() {
@Override
- public Tuple2<Integer, Integer> call(Integer i) throws Exception {
+ public Tuple2<Integer, Integer> call(Integer i) {
return new Tuple2<Integer, Integer>(i % 10, 1);
}
});
JavaPairDStream<Integer, Integer> reducedStream = mappedStream.reduceByKey(
new Function2<Integer, Integer, Integer>() {
@Override
- public Integer call(Integer i1, Integer i2) throws Exception {
+ public Integer call(Integer i1, Integer i2) {
return i1 + i2;
}
});
diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
index a119980992..0097dade19 100644
--- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala
@@ -33,7 +33,7 @@ object BroadcastTest {
System.setProperty("spark.broadcast.blockSize", blockSize)
val sc = new SparkContext(args(0), "Broadcast Test",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val slices = if (args.length > 1) args(1).toInt else 2
val num = if (args.length > 2) args(2).toInt else 1000000
diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
index 92eb96bd8e..b3eb611dd2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala
@@ -27,7 +27,7 @@ object ExceptionHandlingTest {
}
val sc = new SparkContext(args(0), "ExceptionHandlingTest",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
sc.parallelize(0 until sc.defaultParallelism).foreach { i =>
if (math.random > 0.75)
throw new Exception("Testing exception handling")
diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
index 42c2e0e8e1..39752fdd0e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala
@@ -34,7 +34,7 @@ object GroupByTest {
var numReducers = if (args.length > 4) args(4).toInt else numMappers
val sc = new SparkContext(args(0), "GroupBy Test",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
val ranGen = new Random
diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
index efe2e93b0d..65d67356be 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.mapreduce.TableInputFormat
object HBaseTest {
def main(args: Array[String]) {
val sc = new SparkContext(args(0), "HBaseTest",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val conf = HBaseConfiguration.create()
diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
index d6a88d3032..c3597d94a2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala
@@ -22,7 +22,7 @@ import org.apache.spark._
object HdfsTest {
def main(args: Array[String]) {
val sc = new SparkContext(args(0), "HdfsTest",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val file = sc.textFile(args(1))
val mapped = file.map(s => s.length).cache()
for (iter <- 1 to 10) {
diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
index 17ff3ce764..bddb54b39c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala
@@ -45,7 +45,7 @@ object LogQuery {
}
val sc = new SparkContext(args(0), "Log Query",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val dataSet =
if (args.length == 2) sc.textFile(args(1))
diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index e1afc29f9a..4aef04fc06 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -28,7 +28,7 @@ object MultiBroadcastTest {
}
val sc = new SparkContext(args(0), "Multi-Broadcast Test",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val slices = if (args.length > 1) args(1).toInt else 2
val num = if (args.length > 2) args(2).toInt else 1000000
diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
index 37ddfb5db7..73b0e216ca 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala
@@ -36,7 +36,7 @@ object SimpleSkewedGroupByTest {
var ratio = if (args.length > 5) args(5).toInt else 5.0
val sc = new SparkContext(args(0), "GroupBy Test",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
val ranGen = new Random
diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
index 9c954b2b5b..31c6d108f3 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala
@@ -34,7 +34,7 @@ object SkewedGroupByTest {
var numReducers = if (args.length > 4) args(4).toInt else numMappers
val sc = new SparkContext(args(0), "GroupBy Test",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p =>
val ranGen = new Random
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
index 814944ba1c..30c86d83e6 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
@@ -112,7 +112,7 @@ object SparkALS {
printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS)
val sc = new SparkContext(host, "SparkALS",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val R = generateR()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
index 86dd9ca1b3..ff72532db1 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala
@@ -54,7 +54,7 @@ object SparkHdfsLR {
val inputPath = args(1)
val conf = SparkHadoopUtil.get.newConfiguration()
val sc = new SparkContext(args(0), "SparkHdfsLR",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")), Map(),
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(),
InputFormatInfo.computePreferredLocations(
Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath))))
val lines = sc.textFile(inputPath)
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 bc2db39c12..8c99025eaa 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala
@@ -55,7 +55,7 @@ object SparkKMeans {
System.exit(1)
}
val sc = new SparkContext(args(0), "SparkLocalKMeans",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val lines = sc.textFile(args(1))
val data = lines.map(parseVector _).cache()
val K = args(2).toInt
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
index 9ed9fe4d76..c54a55bdb4 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala
@@ -49,7 +49,7 @@ object SparkLR {
System.exit(1)
}
val sc = new SparkContext(args(0), "SparkLR",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val numSlices = if (args.length > 1) args(1).toInt else 2
val points = sc.parallelize(generateData, numSlices).cache()
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
index a508c0df57..d203f4d20e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala
@@ -38,7 +38,7 @@ object SparkPageRank {
}
var iters = args(2).toInt
val ctx = new SparkContext(args(0), "PageRank",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val lines = ctx.textFile(args(1), 1)
val links = lines.map{ s =>
val parts = s.split("\\s+")
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
index a689e5a360..e5a09ecec0 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
@@ -29,7 +29,7 @@ object SparkPi {
System.exit(1)
}
val spark = new SparkContext(args(0), "SparkPi",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val slices = if (args.length > 1) args(1).toInt else 2
val n = 100000 * slices
val count = spark.parallelize(1 to n, slices).map { i =>
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
index 8543ce0e32..24e8afa26b 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala
@@ -46,7 +46,7 @@ object SparkTC {
System.exit(1)
}
val spark = new SparkContext(args(0), "SparkTC",
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
val slices = if (args.length > 1) args(1).toInt else 2
var tc = spark.parallelize(generateGraph, slices).cache()
diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
index 72b5c7b88e..4c0de46964 100644
--- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala
@@ -36,16 +36,18 @@ object WikipediaPageRank {
System.err.println("Usage: WikipediaPageRank <inputFile> <threshold> <numPartitions> <host> <usePartitioner>")
System.exit(-1)
}
-
- System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
- System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName)
+ val sparkConf = new SparkConf()
+ sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+ sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName)
val inputFile = args(0)
val threshold = args(1).toDouble
val numPartitions = args(2).toInt
val host = args(3)
val usePartitioner = args(4).toBoolean
- val sc = new SparkContext(host, "WikipediaPageRank")
+
+ sparkConf.setMaster(host).setAppName("WikipediaPageRank")
+ val sc = new SparkContext(sparkConf)
// Parse the Wikipedia page data into a graph
val input = sc.textFile(inputFile)
diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
index ddf6855325..2cf273a702 100644
--- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala
@@ -34,15 +34,19 @@ object WikipediaPageRankStandalone {
System.err.println("Usage: WikipediaPageRankStandalone <inputFile> <threshold> <numIterations> <host> <usePartitioner>")
System.exit(-1)
}
+ val sparkConf = new SparkConf()
+ sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer")
- System.setProperty("spark.serializer", "spark.bagel.examples.WPRSerializer")
val inputFile = args(0)
val threshold = args(1).toDouble
val numIterations = args(2).toInt
val host = args(3)
val usePartitioner = args(4).toBoolean
- val sc = new SparkContext(host, "WikipediaPageRankStandalone")
+
+ sparkConf.setMaster(host).setAppName("WikipediaPageRankStandalone")
+
+ val sc = new SparkContext(sparkConf)
val input = sc.textFile(inputFile)
val partitioner = new HashPartitioner(sc.defaultParallelism)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
index af52b7e9a1..4e0058cd70 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
@@ -19,12 +19,14 @@ package org.apache.spark.streaming.examples
import scala.collection.mutable.LinkedList
import scala.util.Random
+import scala.reflect.ClassTag
import akka.actor.Actor
import akka.actor.ActorRef
import akka.actor.Props
import akka.actor.actorRef2Scala
+import org.apache.spark.SparkConf
import org.apache.spark.streaming.Seconds
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions
@@ -82,10 +84,10 @@ class FeederActor extends Actor {
*
* @see [[org.apache.spark.streaming.examples.FeederActor]]
*/
-class SampleActorReceiver[T: ClassManifest](urlOfPublisher: String)
+class SampleActorReceiver[T: ClassTag](urlOfPublisher: String)
extends Actor with Receiver {
- lazy private val remotePublisher = context.actorFor(urlOfPublisher)
+ lazy private val remotePublisher = context.actorSelection(urlOfPublisher)
override def preStart = remotePublisher ! SubscribeReceiver(context.self)
@@ -115,7 +117,7 @@ object FeederActor {
val Seq(host, port) = args.toSeq
- val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1
+ val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = new SparkConf)._1
val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor")
println("Feeder started as:" + feeder)
@@ -132,9 +134,9 @@ object FeederActor {
* <hostname> and <port> describe the AkkaSystem that Spark Sample feeder is running on.
*
* To run this example locally, you may run Feeder Actor as
- * `$ ./run-example spark.streaming.examples.FeederActor 127.0.1.1 9999`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.FeederActor 127.0.1.1 9999`
* and then run the example
- * `$ ./run-example spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
*/
object ActorWordCount {
def main(args: Array[String]) {
@@ -149,7 +151,7 @@ object ActorWordCount {
// Create the context and set the batch size
val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
/*
* Following is the use of actorStream to plug in custom actor as receiver
@@ -164,7 +166,7 @@ object ActorWordCount {
*/
val lines = ssc.actorStream[String](
- Props(new SampleActorReceiver[String]("akka://test@%s:%s/user/FeederActor".format(
+ Props(new SampleActorReceiver[String]("akka.tcp://test@%s:%s/user/FeederActor".format(
host, port.toInt))), "SampleReceiver")
//compute wordcount
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
index 9f6e163454..ae3709b3d9 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
@@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples
import org.apache.spark.util.IntParam
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming._
+import org.apache.spark.streaming.flume._
/**
* Produces a count of events received from Flume.
@@ -48,10 +49,10 @@ object FlumeEventCount {
val batchInterval = Milliseconds(2000)
// Create the context and set the batch size
val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval,
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
// Create a flume stream
- val stream = ssc.flumeStream(host,port,StorageLevel.MEMORY_ONLY)
+ val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2)
// Print out the count of events received from this server in each batch
stream.count().map(cnt => "Received " + cnt + " flume events." ).print()
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
index bc8564b3ba..ea6ea67419 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
@@ -28,7 +28,7 @@ import org.apache.spark.streaming.StreamingContext._
* <directory> is the directory that Spark Streaming will use to find and read new text files.
*
* To run this on your local machine on directory `localdir`, run this example
- * `$ ./run-example spark.streaming.examples.HdfsWordCount local[2] localdir`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.HdfsWordCount local[2] localdir`
* Then create a text file in `localdir` and the words in the file will get counted.
*/
object HdfsWordCount {
@@ -40,7 +40,7 @@ object HdfsWordCount {
// Create the context
val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
// Create the FileInputDStream on the directory and use the
// stream to count words in new files created
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
index 570ba4c81a..31a94bd224 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
@@ -24,6 +24,7 @@ import kafka.producer._
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
import org.apache.spark.streaming.util.RawTextHelper._
+import org.apache.spark.streaming.kafka._
/**
* Consumes messages from one or more topics in Kafka and does wordcount.
@@ -35,7 +36,7 @@ import org.apache.spark.streaming.util.RawTextHelper._
* <numThreads> is the number of threads the kafka consumer should use
*
* Example:
- * `./run-example spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
+ * `./bin/run-example org.apache.spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
*/
object KafkaWordCount {
def main(args: Array[String]) {
@@ -48,11 +49,11 @@ object KafkaWordCount {
val Array(master, zkQuorum, group, topics, numThreads) = args
val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
ssc.checkpoint("checkpoint")
val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap
- val lines = ssc.kafkaStream(zkQuorum, group, topicpMap).map(_._2)
+ val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1l))
.reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
index ff332a0282..325290b66f 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
@@ -17,11 +17,6 @@
package org.apache.spark.streaming.examples
-import org.apache.spark.streaming.{ Seconds, StreamingContext }
-import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.streaming.dstream.MQTTReceiver
-import org.apache.spark.storage.StorageLevel
-
import org.eclipse.paho.client.mqttv3.MqttClient
import org.eclipse.paho.client.mqttv3.MqttClientPersistence
import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence
@@ -29,6 +24,11 @@ import org.eclipse.paho.client.mqttv3.MqttException
import org.eclipse.paho.client.mqttv3.MqttMessage
import org.eclipse.paho.client.mqttv3.MqttTopic
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.mqtt._
+
/**
* A simple Mqtt publisher for demonstration purposes, repeatedly publishes
* Space separated String Message "hello mqtt demo for spark streaming"
@@ -79,9 +79,9 @@ object MQTTPublisher {
* <MqttbrokerUrl> and <topic> describe where Mqtt publisher is running.
*
* To run this example locally, you may run publisher as
- * `$ ./run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
* and run the example as
- * `$ ./run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
*/
object MQTTWordCount {
@@ -96,8 +96,8 @@ object MQTTWordCount {
val Seq(master, brokerUrl, topic) = args.toSeq
val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"),
- Seq(System.getenv("SPARK_EXAMPLES_JAR")))
- val lines = ssc.mqttStream(brokerUrl, topic, StorageLevel.MEMORY_ONLY)
+ StreamingContext.jarOfClass(this.getClass))
+ val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2)
val words = lines.flatMap(x => x.toString.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
index e2487dca5f..6a32c75373 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
- * `$ ./run-example spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
*/
object NetworkWordCount {
def main(args: Array[String]) {
@@ -41,7 +41,7 @@ object NetworkWordCount {
// Create the context with a 1 second batch size
val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
// Create a NetworkInputDStream on target ip:port and count the
// words in input stream of \n delimited test (eg. generated by 'nc')
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
index fad512eeba..9d640e716b 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
@@ -33,7 +33,7 @@ object QueueStream {
// Create the context
val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
// Create the queue through which RDDs can be pushed to
// a QueueInputDStream
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
index 0b45c30d20..c0706d0724 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
@@ -49,7 +49,7 @@ object RawNetworkGrep {
// Create the context
val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
// Warm up the JVMs on master and slave for JIT compilation to kick in
RawTextHelper.warmUp(ssc.sparkContext)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
index cb30c4edb3..002db57d59 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
@@ -29,7 +29,7 @@ import org.apache.spark.streaming.StreamingContext._
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
- * `$ ./run-example spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
*/
object StatefulNetworkWordCount {
def main(args: Array[String]) {
@@ -49,7 +49,7 @@ object StatefulNetworkWordCount {
// Create the context with a 1 second batch size
val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", Seconds(1),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
ssc.checkpoint(".")
// Create a NetworkInputDStream on target ip:port and count the
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
index 35b6329ab3..3ccdc908e2 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
@@ -23,6 +23,8 @@ import com.twitter.algebird._
import org.apache.spark.streaming.StreamingContext._
import org.apache.spark.SparkContext._
+import org.apache.spark.streaming.twitter._
+
/**
* Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute
* windowed and global Top-K estimates of user IDs occurring in a Twitter stream.
@@ -33,7 +35,7 @@ import org.apache.spark.SparkContext._
* <p>
* <p>
* <a href="http://highlyscalable.wordpress.com/2012/05/01/probabilistic-structures-web-analytics-data-mining/">
- * This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a datastructure
+ * This blog post</a> has a good overview of the Count-Min Sketch (CMS). The CMS is a data structure
* for approximate frequency estimation in data streams (e.g. Top-K elements, frequency of any given element, etc),
* that uses space sub-linear in the number of elements in the stream. Once elements are added to the CMS, the
* estimated count of an element can be computed, as well as "heavy-hitters" that occur more than a threshold
@@ -60,8 +62,8 @@ object TwitterAlgebirdCMS {
val (master, filters) = (args.head, args.tail)
val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
- val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER)
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
+ val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2)
val users = stream.map(status => status.getUser.getId)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
index 8bfde2a829..c7e83e76b0 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
@@ -21,7 +21,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.storage.StorageLevel
import com.twitter.algebird.HyperLogLog._
import com.twitter.algebird.HyperLogLogMonoid
-import org.apache.spark.streaming.dstream.TwitterInputDStream
+import org.apache.spark.streaming.twitter._
/**
* Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute
@@ -49,8 +49,8 @@ object TwitterAlgebirdHLL {
val (master, filters) = (args.head, args.tail)
val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
- val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER)
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
+ val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER)
val users = stream.map(status => status.getUser.getId)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
index 27aa6b14bf..e2b0418d55 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
@@ -20,6 +20,7 @@ package org.apache.spark.streaming.examples
import org.apache.spark.streaming.{Seconds, StreamingContext}
import StreamingContext._
import org.apache.spark.SparkContext._
+import org.apache.spark.streaming.twitter._
/**
* Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter
@@ -38,8 +39,8 @@ object TwitterPopularTags {
val (master, filters) = (args.head, args.tail)
val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
- val stream = ssc.twitterStream(None, filters)
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
+ val stream = TwitterUtils.createStream(ssc, None, filters)
val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#")))
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
index c8743b9e25..03902ec353 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
@@ -20,9 +20,12 @@ package org.apache.spark.streaming.examples
import akka.actor.ActorSystem
import akka.actor.actorRef2Scala
import akka.zeromq._
-import org.apache.spark.streaming.{ Seconds, StreamingContext }
-import org.apache.spark.streaming.StreamingContext._
import akka.zeromq.Subscribe
+import akka.util.ByteString
+
+import org.apache.spark.streaming.{Seconds, StreamingContext}
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.zeromq._
/**
* A simple publisher for demonstration purposes, repeatedly publishes random Messages
@@ -40,10 +43,11 @@ object SimpleZeroMQPublisher {
val acs: ActorSystem = ActorSystem()
val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url))
- val messages: Array[String] = Array("words ", "may ", "count ")
+ implicit def stringToByteString(x: String) = ByteString(x)
+ val messages: List[ByteString] = List("words ", "may ", "count ")
while (true) {
Thread.sleep(1000)
- pubSocket ! ZMQMessage(Frame(topic) :: messages.map(x => Frame(x.getBytes)).toList)
+ pubSocket ! ZMQMessage(ByteString(topic) :: messages)
}
acs.awaitTermination()
}
@@ -60,9 +64,9 @@ object SimpleZeroMQPublisher {
* <zeroMQurl> and <topic> describe where zeroMq publisher is running.
*
* To run this example locally, you may run publisher as
- * `$ ./run-example spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
* and run the example as
- * `$ ./run-example spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
+ * `$ ./bin/run-example org.apache.spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
*/
object ZeroMQWordCount {
def main(args: Array[String]) {
@@ -76,16 +80,15 @@ object ZeroMQWordCount {
// Create the context and set the batch size
val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
- def bytesToStringIterator(x: Seq[Seq[Byte]]) = (x.map(x => new String(x.toArray))).iterator
+ def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator
//For this stream, a zeroMQ publisher should be running.
- val lines = ssc.zeroMQStream(url, Subscribe(topic), bytesToStringIterator)
+ val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
}
-
}
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
index de70c50473..4fe57de4a4 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
@@ -39,8 +39,8 @@ object PageView extends Serializable {
/** Generates streaming events to simulate page views on a website.
*
* This should be used in tandem with PageViewStream.scala. Example:
- * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
- * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+ * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+ * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
*
* When running this, you may want to set the root logging level to ERROR in
* conf/log4j.properties to reduce the verbosity of the output.
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
index 8282cc9269..807af199f4 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
@@ -25,8 +25,8 @@ import org.apache.spark.SparkContext._
* operators available in Spark streaming.
*
* This should be used in tandem with PageViewStream.scala. Example:
- * $ ./run-example spark.streaming.examples.clickstream.PageViewGenerator 44444 10
- * $ ./run-example spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+ * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10
+ * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
*/
object PageViewStream {
def main(args: Array[String]) {
@@ -42,7 +42,7 @@ object PageViewStream {
// Create the context
val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1),
- System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
+ System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass))
// Create a NetworkInputDStream on target host:port and convert each line to a PageView
val pageViews = ssc.socketTextStream(host, port)
diff --git a/external/flume/pom.xml b/external/flume/pom.xml
new file mode 100644
index 0000000000..443910a03a
--- /dev/null
+++ b/external/flume/pom.xml
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-parent</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-flume_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project External Flume</name>
+ <url>http://spark.incubator.apache.org/</url>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flume</groupId>
+ <artifactId>flume-ng-sdk</artifactId>
+ <version>1.2.0</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>org.xerial.snappy</groupId>
+ <artifactId>*</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.scalacheck</groupId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.novocode</groupId>
+ <artifactId>junit-interface</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+ <plugins>
+ <plugin>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest-maven-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
index a0189eca04..ce3ef47cfe 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlumeInputDStream.scala
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
@@ -15,13 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.flume
import java.net.InetSocketAddress
import java.io.{ObjectInput, ObjectOutput, Externalizable}
import java.nio.ByteBuffer
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.flume.source.avro.AvroSourceProtocol
import org.apache.flume.source.avro.AvroFlumeEvent
@@ -29,12 +30,13 @@ import org.apache.flume.source.avro.Status
import org.apache.avro.ipc.specific.SpecificResponder
import org.apache.avro.ipc.NettyServer
-import org.apache.spark.streaming.StreamingContext
import org.apache.spark.util.Utils
import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
private[streaming]
-class FlumeInputDStream[T: ClassManifest](
+class FlumeInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
host: String,
port: Int,
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
new file mode 100644
index 0000000000..834b775d4f
--- /dev/null
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.flume
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+
+object FlumeUtils {
+ /**
+ * Create a input stream from a Flume source.
+ * @param ssc StreamingContext object
+ * @param hostname Hostname of the slave machine to which the flume data will be sent
+ * @param port Port of the slave machine to which the flume data will be sent
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def createStream (
+ ssc: StreamingContext,
+ hostname: String,
+ port: Int,
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+ ): DStream[SparkFlumeEvent] = {
+ val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel)
+ ssc.registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Creates a input stream from a Flume source.
+ * @param hostname Hostname of the slave machine to which the flume data will be sent
+ * @param port Port of the slave machine to which the flume data will be sent
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ hostname: String,
+ port: Int
+ ): JavaDStream[SparkFlumeEvent] = {
+ createStream(jssc.ssc, hostname, port)
+ }
+
+ /**
+ * Creates a input stream from a Flume source.
+ * @param hostname Hostname of the slave machine to which the flume data will be sent
+ * @param port Port of the slave machine to which the flume data will be sent
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ hostname: String,
+ port: Int,
+ storageLevel: StorageLevel
+ ): JavaDStream[SparkFlumeEvent] = {
+ createStream(jssc.ssc, hostname, port, storageLevel)
+ }
+}
diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
new file mode 100644
index 0000000000..733389b98d
--- /dev/null
+++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java
@@ -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.streaming.flume;
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+import org.junit.Test;
+
+public class JavaFlumeStreamSuite extends LocalJavaStreamingContext {
+ @Test
+ public void testFlumeStream() {
+ // tests the API, does not actually test data receiving
+ JavaDStream<SparkFlumeEvent> test1 = FlumeUtils.createStream(ssc, "localhost", 12345);
+ JavaDStream<SparkFlumeEvent> test2 = FlumeUtils.createStream(ssc, "localhost", 12345,
+ StorageLevel.MEMORY_AND_DISK_SER_2());
+ }
+}
diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..063529a9cb
--- /dev/null
+++ b/external/flume/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
new file mode 100644
index 0000000000..2e8e9fac45
--- /dev/null
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.flume
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
+
+import java.net.InetSocketAddress
+import java.nio.ByteBuffer
+import java.nio.charset.Charset
+
+import org.apache.avro.ipc.NettyTransceiver
+import org.apache.avro.ipc.specific.SpecificRequestor
+import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol}
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase}
+import org.apache.spark.streaming.util.ManualClock
+
+class FlumeStreamSuite extends TestSuiteBase {
+
+ val testPort = 9999
+
+ test("flume input stream") {
+ // Set up the streaming context and input streams
+ val ssc = new StreamingContext(conf, batchDuration)
+ val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK)
+ val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
+ with SynchronizedBuffer[Seq[SparkFlumeEvent]]
+ val outputStream = new TestOutputStream(flumeStream, outputBuffer)
+ ssc.registerOutputStream(outputStream)
+ ssc.start()
+
+ val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
+ val input = Seq(1, 2, 3, 4, 5)
+ Thread.sleep(1000)
+ val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
+ val client = SpecificRequestor.getClient(
+ classOf[AvroSourceProtocol], transceiver)
+
+ for (i <- 0 until input.size) {
+ val event = new AvroFlumeEvent
+ event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
+ event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
+ client.append(event)
+ Thread.sleep(500)
+ clock.addToTime(batchDuration.milliseconds)
+ }
+
+ val startTime = System.currentTimeMillis()
+ while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
+ logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
+ Thread.sleep(100)
+ }
+ Thread.sleep(1000)
+ val timeTaken = System.currentTimeMillis() - startTime
+ assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
+ logInfo("Stopping context")
+ ssc.stop()
+
+ val decoder = Charset.forName("UTF-8").newDecoder()
+
+ assert(outputBuffer.size === input.length)
+ for (i <- 0 until outputBuffer.size) {
+ assert(outputBuffer(i).size === 1)
+ val str = decoder.decode(outputBuffer(i).head.event.getBody)
+ assert(str.toString === input(i).toString)
+ assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
+ }
+ }
+}
diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml
new file mode 100644
index 0000000000..f782e0e126
--- /dev/null
+++ b/external/kafka/pom.xml
@@ -0,0 +1,97 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-parent</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-kafka_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project External Kafka</name>
+ <url>http://spark.incubator.apache.org/</url>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.sksamuel.kafka</groupId>
+ <artifactId>kafka_${scala.binary.version}</artifactId>
+ <version>0.8.0-beta1</version>
+ <exclusions>
+ <exclusion>
+ <groupId>com.sun.jmx</groupId>
+ <artifactId>jmxri</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.sun.jdmk</groupId>
+ <artifactId>jmxtools</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>net.sf.jopt-simple</groupId>
+ <artifactId>jopt-simple</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.scalacheck</groupId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.novocode</groupId>
+ <artifactId>junit-interface</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+ <plugins>
+ <plugin>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest-maven-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
index a5de5e1fb5..a2cd49c573 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/KafkaInputDStream.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala
@@ -15,11 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.kafka
-import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.StreamingContext
+import scala.collection.Map
+import scala.reflect.ClassTag
import java.util.Properties
import java.util.concurrent.Executors
@@ -30,12 +29,14 @@ import kafka.utils.VerifiableProperties
import kafka.utils.ZKStringSerializer
import org.I0Itec.zkclient._
-import scala.collection.Map
-
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
/**
* Input stream that pulls messages from a Kafka Broker.
- *
+ *
* @param kafkaParams Map of kafka configuration paramaters. See: http://kafka.apache.org/configuration.html
* @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
* in its own thread.
@@ -43,8 +44,8 @@ import scala.collection.Map
*/
private[streaming]
class KafkaInputDStream[
- K: ClassManifest,
- V: ClassManifest,
+ K: ClassTag,
+ V: ClassTag,
U <: Decoder[_]: Manifest,
T <: Decoder[_]: Manifest](
@transient ssc_ : StreamingContext,
@@ -61,8 +62,8 @@ class KafkaInputDStream[
private[streaming]
class KafkaReceiver[
- K: ClassManifest,
- V: ClassManifest,
+ K: ClassTag,
+ V: ClassTag,
U <: Decoder[_]: Manifest,
T <: Decoder[_]: Manifest](
kafkaParams: Map[String, String],
@@ -104,17 +105,18 @@ class KafkaReceiver[
tryZookeeperConsumerGroupCleanup(kafkaParams("zookeeper.connect"), kafkaParams("group.id"))
}
- // Create Threads for each Topic/Message Stream we are listening
- val keyDecoder = manifest[U].erasure.getConstructor(classOf[VerifiableProperties])
+ val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
.newInstance(consumerConfig.props)
.asInstanceOf[Decoder[K]]
- val valueDecoder = manifest[T].erasure.getConstructor(classOf[VerifiableProperties])
+ val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties])
.newInstance(consumerConfig.props)
.asInstanceOf[Decoder[V]]
+ // Create Threads for each Topic/Message Stream we are listening
val topicMessageStreams = consumerConnector.createMessageStreams(
topics, keyDecoder, valueDecoder)
+
// Start the messages handler for each partition
topicMessageStreams.values.foreach { streams =>
streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) }
@@ -122,7 +124,7 @@ class KafkaReceiver[
}
// Handles Kafka Messages
- private class MessageHandler[K: ClassManifest, V: ClassManifest](stream: KafkaStream[K, V])
+ private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V])
extends Runnable {
def run() {
logInfo("Starting MessageHandler.")
@@ -146,7 +148,7 @@ class KafkaReceiver[
zk.deleteRecursive(dir)
zk.close()
} catch {
- case _ => // swallow
+ case _ : Throwable => // swallow
}
}
}
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
new file mode 100644
index 0000000000..c2d851f943
--- /dev/null
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+
+import java.lang.{Integer => JInt}
+import java.util.{Map => JMap}
+
+import kafka.serializer.{Decoder, StringDecoder}
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream}
+
+
+object KafkaUtils {
+ /**
+ * Create an input stream that pulls messages from a Kafka Broker.
+ * @param ssc StreamingContext object
+ * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..)
+ * @param groupId The group id for this consumer
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread
+ * @param storageLevel Storage level to use for storing the received objects
+ * (default: StorageLevel.MEMORY_AND_DISK_SER_2)
+ */
+ def createStream(
+ ssc: StreamingContext,
+ zkQuorum: String,
+ groupId: String,
+ topics: Map[String, Int],
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+ ): DStream[(String, String)] = {
+ val kafkaParams = Map[String, String](
+ "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
+ "zookeeper.connection.timeout.ms" -> "10000")
+ createStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, topics, storageLevel)
+ }
+
+ /**
+ * Create an input stream that pulls messages from a Kafka Broker.
+ * @param ssc StreamingContext object
+ * @param kafkaParams Map of kafka configuration parameters,
+ * see http://kafka.apache.org/08/configuration.html
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest](
+ ssc: StreamingContext,
+ kafkaParams: Map[String, String],
+ topics: Map[String, Int],
+ storageLevel: StorageLevel
+ ): DStream[(K, V)] = {
+ val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
+ ssc.registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Create an input stream that pulls messages form a Kafka Broker.
+ * @param jssc JavaStreamingContext object
+ * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..)
+ * @param groupId The group id for this consumer
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ zkQuorum: String,
+ groupId: String,
+ topics: JMap[String, JInt]
+ ): JavaPairDStream[String, String] = {
+ implicit val cmt: ClassTag[String] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+ createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
+ }
+
+ /**
+ * Create an input stream that pulls messages form a Kafka Broker.
+ * @param jssc JavaStreamingContext object
+ * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..).
+ * @param groupId The group id for this consumer.
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread.
+ * @param storageLevel RDD storage level.
+ *
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ zkQuorum: String,
+ groupId: String,
+ topics: JMap[String, JInt],
+ storageLevel: StorageLevel
+ ): JavaPairDStream[String, String] = {
+ implicit val cmt: ClassTag[String] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+ createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+ }
+
+ /**
+ * Create an input stream that pulls messages form a Kafka Broker.
+ * @param jssc JavaStreamingContext object
+ * @param keyTypeClass Key type of RDD
+ * @param valueTypeClass value type of RDD
+ * @param keyDecoderClass Type of kafka key decoder
+ * @param valueDecoderClass Type of kafka value decoder
+ * @param kafkaParams Map of kafka configuration parameters,
+ * see http://kafka.apache.org/08/configuration.html
+ * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
+ * in its own thread
+ * @param storageLevel RDD storage level. Defaults to MEMORY_AND_DISK_2.
+ */
+ def createStream[K, V, U <: Decoder[_], T <: Decoder[_]](
+ jssc: JavaStreamingContext,
+ keyTypeClass: Class[K],
+ valueTypeClass: Class[V],
+ keyDecoderClass: Class[U],
+ valueDecoderClass: Class[T],
+ kafkaParams: JMap[String, String],
+ topics: JMap[String, JInt],
+ storageLevel: StorageLevel
+ ): JavaPairDStream[K, V] = {
+ implicit val keyCmt: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val valueCmt: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
+
+ implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
+ implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
+
+ createStream[K, V, U, T](
+ jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel)
+ }
+}
diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
new file mode 100644
index 0000000000..7b4999447e
--- /dev/null
+++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java
@@ -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.streaming.kafka;
+
+import java.util.HashMap;
+import org.junit.Test;
+import com.google.common.collect.Maps;
+import kafka.serializer.StringDecoder;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaPairDStream;
+
+public class JavaKafkaStreamSuite extends LocalJavaStreamingContext {
+ @Test
+ public void testKafkaStream() {
+ HashMap<String, Integer> topics = Maps.newHashMap();
+
+ // tests the API, does not actually test data receiving
+ JavaPairDStream<String, String> test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics);
+ JavaPairDStream<String, String> test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics,
+ StorageLevel.MEMORY_AND_DISK_SER_2());
+
+ HashMap<String, String> kafkaParams = Maps.newHashMap();
+ kafkaParams.put("zookeeper.connect", "localhost:12345");
+ kafkaParams.put("group.id","consumer-group");
+ JavaPairDStream<String, String> test3 = KafkaUtils.createStream(ssc,
+ String.class, String.class, StringDecoder.class, StringDecoder.class,
+ kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2());
+ }
+}
diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..063529a9cb
--- /dev/null
+++ b/external/kafka/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
new file mode 100644
index 0000000000..9c81f23c19
--- /dev/null
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.kafka
+
+import kafka.serializer.StringDecoder
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+
+class KafkaStreamSuite extends TestSuiteBase {
+
+ test("kafka input stream") {
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ val topics = Map("my-topic" -> 1)
+
+ // tests the API, does not actually test data receiving
+ val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics)
+ val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+ val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
+ val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
+ ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2)
+
+ // TODO: Actually test receiving data
+ }
+}
diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml
new file mode 100644
index 0000000000..31b4fa87de
--- /dev/null
+++ b/external/mqtt/pom.xml
@@ -0,0 +1,108 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-parent</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-mqtt_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project External MQTT</name>
+ <url>http://spark.incubator.apache.org/</url>
+
+ <repositories>
+ <repository>
+ <id>mqtt-repo</id>
+ <name>MQTT Repository</name>
+ <url>https://repo.eclipse.org/content/repositories/paho-releases</url>
+ <releases>
+ <enabled>true</enabled>
+ </releases>
+ <snapshots>
+ <enabled>false</enabled>
+ </snapshots>
+ </repository>
+ </repositories>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.eclipse.paho</groupId>
+ <artifactId>mqtt-client</artifactId>
+ <version>0.4.0</version>
+ </dependency>
+ <dependency>
+ <groupId>${akka.group}</groupId>
+ <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
+ <version>${akka.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.scalacheck</groupId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.novocode</groupId>
+ <artifactId>junit-interface</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+ <plugins>
+ <plugin>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest-maven-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
index ac0528213d..c8987a3ee0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MQTTInputDStream.scala
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala
@@ -15,11 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.mqtt
-import org.apache.spark.Logging
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{ Time, DStreamCheckpointData, StreamingContext }
+import scala.collection.Map
+import scala.collection.mutable.HashMap
+import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import java.util.Properties
import java.util.concurrent.Executors
@@ -34,9 +35,10 @@ import org.eclipse.paho.client.mqttv3.MqttException
import org.eclipse.paho.client.mqttv3.MqttMessage
import org.eclipse.paho.client.mqttv3.MqttTopic
-import scala.collection.Map
-import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
+import org.apache.spark.Logging
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.StreamingContext
+import org.apache.spark.streaming.dstream._
/**
* Input stream that subscribe messages from a Mqtt Broker.
@@ -47,16 +49,15 @@ import scala.collection.JavaConversions._
*/
private[streaming]
-class MQTTInputDStream[T: ClassManifest](
- @transient ssc_ : StreamingContext,
- brokerUrl: String,
- topic: String,
- storageLevel: StorageLevel
+class MQTTInputDStream[T: ClassTag](
+ @transient ssc_ : StreamingContext,
+ brokerUrl: String,
+ topic: String,
+ storageLevel: StorageLevel
) extends NetworkInputDStream[T](ssc_) with Logging {
def getReceiver(): NetworkReceiver[T] = {
- new MQTTReceiver(brokerUrl, topic, storageLevel)
- .asInstanceOf[NetworkReceiver[T]]
+ new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]]
}
}
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
new file mode 100644
index 0000000000..0e6c25dbee
--- /dev/null
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+import scala.reflect.ClassTag
+
+object MQTTUtils {
+ /**
+ * Create an input stream that receives messages pushed by a MQTT publisher.
+ * @param ssc StreamingContext object
+ * @param brokerUrl Url of remote MQTT publisher
+ * @param topic Topic name to subscribe to
+ * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+ */
+ def createStream(
+ ssc: StreamingContext,
+ brokerUrl: String,
+ topic: String,
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+ ): DStream[String] = {
+ val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
+ ssc.registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Create an input stream that receives messages pushed by a MQTT publisher.
+ * @param jssc JavaStreamingContext object
+ * @param brokerUrl Url of remote MQTT publisher
+ * @param topic Topic name to subscribe to
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ brokerUrl: String,
+ topic: String
+ ): JavaDStream[String] = {
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+ createStream(jssc.ssc, brokerUrl, topic)
+ }
+
+ /**
+ * Create an input stream that receives messages pushed by a MQTT publisher.
+ * @param jssc JavaStreamingContext object
+ * @param brokerUrl Url of remote MQTT publisher
+ * @param topic Topic name to subscribe to
+ * @param storageLevel RDD storage level.
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ brokerUrl: String,
+ topic: String,
+ storageLevel: StorageLevel
+ ): JavaDStream[String] = {
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]]
+ createStream(jssc.ssc, brokerUrl, topic, storageLevel)
+ }
+}
diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
new file mode 100644
index 0000000000..44743aaecf
--- /dev/null
+++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt;
+
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.api.java.JavaDStream;
+import org.junit.Test;
+
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+
+public class JavaMQTTStreamSuite extends LocalJavaStreamingContext {
+ @Test
+ public void testMQTTStream() {
+ String brokerUrl = "abc";
+ String topic = "def";
+
+ // tests the API, does not actually test data receiving
+ JavaDStream<String> test1 = MQTTUtils.createStream(ssc, brokerUrl, topic);
+ JavaDStream<String> test2 = MQTTUtils.createStream(ssc, brokerUrl, topic,
+ StorageLevel.MEMORY_AND_DISK_SER_2());
+ }
+}
diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..063529a9cb
--- /dev/null
+++ b/external/mqtt/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
new file mode 100644
index 0000000000..fcc159e85a
--- /dev/null
+++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.mqtt
+
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+
+class MQTTStreamSuite extends TestSuiteBase {
+
+ test("MQTT input stream") {
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ val brokerUrl = "abc"
+ val topic = "def"
+
+ // tests the API, does not actually test data receiving
+ val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic)
+ val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2)
+
+ // TODO: Actually test receiving data
+ }
+}
diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml
new file mode 100644
index 0000000000..216e6c1d8f
--- /dev/null
+++ b/external/twitter/pom.xml
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-parent</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-twitter_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project External Twitter</name>
+ <url>http://spark.incubator.apache.org/</url>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.twitter4j</groupId>
+ <artifactId>twitter4j-stream</artifactId>
+ <version>3.0.3</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.scalacheck</groupId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.novocode</groupId>
+ <artifactId>junit-interface</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+ <plugins>
+ <plugin>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest-maven-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
index 387e15b0e6..5cc721d7f9 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TwitterInputDStream.scala
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala
@@ -15,18 +15,19 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.dstream
+package org.apache.spark.streaming.twitter
-import org.apache.spark._
-import org.apache.spark.streaming._
-import storage.StorageLevel
+import java.util.prefs.Preferences
import twitter4j._
import twitter4j.auth.Authorization
-import java.util.prefs.Preferences
import twitter4j.conf.ConfigurationBuilder
import twitter4j.conf.PropertyConfiguration
import twitter4j.auth.OAuthAuthorization
import twitter4j.auth.AccessToken
+import org.apache.spark._
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.storage.StorageLevel
/* A stream of Twitter statuses, potentially filtered by one or more keywords.
*
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
new file mode 100644
index 0000000000..5e506ffabc
--- /dev/null
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import twitter4j.Status
+import twitter4j.auth.Authorization
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext}
+
+object TwitterUtils {
+ /**
+ * Create a input stream that returns tweets received from Twitter.
+ * @param ssc StreamingContext object
+ * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
+ * authorization; this uses the system properties twitter4j.oauth.consumerKey,
+ * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+ * twitter4j.oauth.accessTokenSecret
+ * @param filters Set of filter strings to get only those tweets that match them
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def createStream(
+ ssc: StreamingContext,
+ twitterAuth: Option[Authorization],
+ filters: Seq[String] = Nil,
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
+ ): DStream[Status] = {
+ val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
+ ssc.registerInputStream(inputStream)
+ inputStream
+ }
+
+ /**
+ * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+ * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+ * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+ * twitter4j.oauth.accessTokenSecret.
+ * @param jssc JavaStreamingContext object
+ */
+ def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = {
+ createStream(jssc.ssc, None)
+ }
+
+ /**
+ * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+ * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+ * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+ * twitter4j.oauth.accessTokenSecret.
+ * @param jssc JavaStreamingContext object
+ * @param filters Set of filter strings to get only those tweets that match them
+ */
+ def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = {
+ createStream(jssc.ssc, None, filters)
+ }
+
+ /**
+ * Create a input stream that returns tweets received from Twitter using Twitter4J's default
+ * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
+ * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and
+ * twitter4j.oauth.accessTokenSecret.
+ * @param jssc JavaStreamingContext object
+ * @param filters Set of filter strings to get only those tweets that match them
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ filters: Array[String],
+ storageLevel: StorageLevel
+ ): JavaDStream[Status] = {
+ createStream(jssc.ssc, None, filters, storageLevel)
+ }
+
+ /**
+ * Create a input stream that returns tweets received from Twitter.
+ * @param jssc JavaStreamingContext object
+ * @param twitterAuth Twitter4J Authorization
+ */
+ def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = {
+ createStream(jssc.ssc, Some(twitterAuth))
+ }
+
+ /**
+ * Create a input stream that returns tweets received from Twitter.
+ * @param jssc JavaStreamingContext object
+ * @param twitterAuth Twitter4J Authorization
+ * @param filters Set of filter strings to get only those tweets that match them
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ twitterAuth: Authorization,
+ filters: Array[String]
+ ): JavaDStream[Status] = {
+ createStream(jssc.ssc, Some(twitterAuth), filters)
+ }
+
+ /**
+ * Create a input stream that returns tweets received from Twitter.
+ * @param jssc JavaStreamingContext object
+ * @param twitterAuth Twitter4J Authorization object
+ * @param filters Set of filter strings to get only those tweets that match them
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def createStream(
+ jssc: JavaStreamingContext,
+ twitterAuth: Authorization,
+ filters: Array[String],
+ storageLevel: StorageLevel
+ ): JavaDStream[Status] = {
+ createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel)
+ }
+}
diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
new file mode 100644
index 0000000000..e46b4e5c75
--- /dev/null
+++ b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java
@@ -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.streaming.twitter;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+import twitter4j.Status;
+import twitter4j.auth.Authorization;
+import twitter4j.auth.NullAuthorization;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+public class JavaTwitterStreamSuite extends LocalJavaStreamingContext {
+ @Test
+ public void testTwitterStream() {
+ String[] filters = (String[])Arrays.<String>asList("filter1", "filter2").toArray();
+ Authorization auth = NullAuthorization.getInstance();
+
+ // tests the API, does not actually test data receiving
+ JavaDStream<Status> test1 = TwitterUtils.createStream(ssc);
+ JavaDStream<Status> test2 = TwitterUtils.createStream(ssc, filters);
+ JavaDStream<Status> test3 = TwitterUtils.createStream(
+ ssc, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+ JavaDStream<Status> test4 = TwitterUtils.createStream(ssc, auth);
+ JavaDStream<Status> test5 = TwitterUtils.createStream(ssc, auth, filters);
+ JavaDStream<Status> test6 = TwitterUtils.createStream(ssc,
+ auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2());
+ }
+}
diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..063529a9cb
--- /dev/null
+++ b/external/twitter/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
new file mode 100644
index 0000000000..a0a8fe617b
--- /dev/null
+++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
@@ -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.
+ */
+
+package org.apache.spark.streaming.twitter
+
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+import org.apache.spark.storage.StorageLevel
+import twitter4j.auth.{NullAuthorization, Authorization}
+
+class TwitterStreamSuite extends TestSuiteBase {
+
+ test("kafka input stream") {
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ val filters = Seq("filter1", "filter2")
+ val authorization: Authorization = NullAuthorization.getInstance()
+
+ // tests the API, does not actually test data receiving
+ val test1 = TwitterUtils.createStream(ssc, None)
+ val test2 = TwitterUtils.createStream(ssc, None, filters)
+ val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2)
+ val test4 = TwitterUtils.createStream(ssc, Some(authorization))
+ val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters)
+ val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters,
+ StorageLevel.MEMORY_AND_DISK_SER_2)
+
+ // Note that actually testing the data receiving is hard as authentication keys are
+ // necessary for accessing Twitter live stream
+ }
+}
diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml
new file mode 100644
index 0000000000..c240d59574
--- /dev/null
+++ b/external/zeromq/pom.xml
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-parent</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../../pom.xml</relativePath>
+ </parent>
+
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming-zeromq_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project External ZeroMQ</name>
+ <url>http://spark.incubator.apache.org/</url>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>${akka.group}</groupId>
+ <artifactId>akka-zeromq_${scala.binary.version}</artifactId>
+ <version>${akka.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.jboss.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.scalacheck</groupId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.novocode</groupId>
+ <artifactId>junit-interface</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+ <plugins>
+ <plugin>
+ <groupId>org.scalatest</groupId>
+ <artifactId>scalatest-maven-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
index 043bb8c8bf..769761e3b8 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ZeroMQReceiver.scala
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala
@@ -15,22 +15,26 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.receivers
+package org.apache.spark.streaming.zeromq
+
+import scala.reflect.ClassTag
import akka.actor.Actor
+import akka.util.ByteString
import akka.zeromq._
import org.apache.spark.Logging
+import org.apache.spark.streaming.receivers._
/**
* A receiver to subscribe to ZeroMQ stream.
*/
-private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String,
+private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String,
subscribe: Subscribe,
- bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T])
+ bytesToObjects: Seq[ByteString] ⇒ Iterator[T])
extends Actor with Receiver with Logging {
- override def preStart() = context.system.newSocket(SocketType.Sub, Listener(self),
+ override def preStart() = ZeroMQExtension(context.system).newSocket(SocketType.Sub, Listener(self),
Connect(publisherUrl), subscribe)
def receive: Receive = {
@@ -38,10 +42,10 @@ private[streaming] class ZeroMQReceiver[T: ClassManifest](publisherUrl: String,
case Connecting ⇒ logInfo("connecting ...")
case m: ZMQMessage ⇒
- logDebug("Received message for:" + m.firstFrameAsString)
+ logDebug("Received message for:" + m.frame(0))
//We ignore first frame for processing as it is the topic
- val bytes = m.frames.tail.map(_.payload)
+ val bytes = m.frames.tail
pushBlock(bytesToObjects(bytes))
case Closed ⇒ logInfo("received closed ")
diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala
new file mode 100644
index 0000000000..546d9df3b5
--- /dev/null
+++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import scala.reflect.ClassTag
+import scala.collection.JavaConversions._
+import akka.actor.{Props, SupervisorStrategy}
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+import org.apache.spark.api.java.function.{Function => JFunction}
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy
+import org.apache.spark.streaming.{StreamingContext, DStream}
+import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream}
+
+object ZeroMQUtils {
+ /**
+ * Create an input stream that receives messages pushed by a zeromq publisher.
+ * @param ssc StreamingContext object
+ * @param publisherUrl Url of remote zeromq publisher
+ * @param subscribe Topic to subscribe to
+ * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
+ * and each frame has sequence of byte thus it needs the converter
+ * (which might be deserializer of bytes) to translate from sequence
+ * of sequence of bytes, where sequence refer to a frame
+ * and sub sequence refer to its payload.
+ * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2.
+ */
+ def createStream[T: ClassTag](
+ ssc: StreamingContext,
+ publisherUrl: String,
+ subscribe: Subscribe,
+ bytesToObjects: Seq[ByteString] ⇒ Iterator[T],
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2,
+ supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
+ ): DStream[T] = {
+ ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)),
+ "ZeroMQReceiver", storageLevel, supervisorStrategy)
+ }
+
+ /**
+ * Create an input stream that receives messages pushed by a zeromq publisher.
+ * @param jssc JavaStreamingContext object
+ * @param publisherUrl Url of remote ZeroMQ publisher
+ * @param subscribe Topic to subscribe to
+ * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+ * of byte thus it needs the converter(which might be deserializer of bytes)
+ * to translate from sequence of sequence of bytes, where sequence refer to a frame
+ * and sub sequence refer to its payload.
+ * @param storageLevel Storage level to use for storing the received objects
+ */
+ def createStream[T](
+ jssc: JavaStreamingContext,
+ publisherUrl: String,
+ subscribe: Subscribe,
+ bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+ storageLevel: StorageLevel,
+ supervisorStrategy: SupervisorStrategy
+ ): JavaDStream[T] = {
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+ val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+ createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy)
+ }
+
+ /**
+ * Create an input stream that receives messages pushed by a zeromq publisher.
+ * @param jssc JavaStreamingContext object
+ * @param publisherUrl Url of remote zeromq publisher
+ * @param subscribe Topic to subscribe to
+ * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+ * of byte thus it needs the converter(which might be deserializer of bytes)
+ * to translate from sequence of sequence of bytes, where sequence refer to a frame
+ * and sub sequence refer to its payload.
+ * @param storageLevel RDD storage level.
+ */
+ def createStream[T](
+ jssc: JavaStreamingContext,
+ publisherUrl: String,
+ subscribe: Subscribe,
+ bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
+ storageLevel: StorageLevel
+ ): JavaDStream[T] = {
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+ val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+ createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel)
+ }
+
+ /**
+ * Create an input stream that receives messages pushed by a zeromq publisher.
+ * @param jssc JavaStreamingContext object
+ * @param publisherUrl Url of remote zeromq publisher
+ * @param subscribe Topic to subscribe to
+ * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
+ * of byte thus it needs the converter(which might be deserializer of bytes)
+ * to translate from sequence of sequence of bytes, where sequence refer to a frame
+ * and sub sequence refer to its payload.
+ */
+ def createStream[T](
+ jssc: JavaStreamingContext,
+ publisherUrl: String,
+ subscribe: Subscribe,
+ bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
+ ): JavaDStream[T] = {
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
+ val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
+ createStream[T](jssc.ssc, publisherUrl, subscribe, fn)
+ }
+}
diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
new file mode 100644
index 0000000000..d2361e14b8
--- /dev/null
+++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq;
+
+import org.junit.Test;
+import akka.actor.SupervisorStrategy;
+import akka.util.ByteString;
+import akka.zeromq.Subscribe;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.storage.StorageLevel;
+import org.apache.spark.streaming.LocalJavaStreamingContext;
+import org.apache.spark.streaming.api.java.JavaDStream;
+
+public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext {
+
+ @Test // tests the API, does not actually test data receiving
+ public void testZeroMQStream() {
+ String publishUrl = "abc";
+ Subscribe subscribe = new Subscribe((ByteString)null);
+ Function<byte[][], Iterable<String>> bytesToObjects = new Function<byte[][], Iterable<String>>() {
+ @Override
+ public Iterable<String> call(byte[][] bytes) throws Exception {
+ return null;
+ }
+ };
+
+ JavaDStream<String> test1 = ZeroMQUtils.<String>createStream(
+ ssc, publishUrl, subscribe, bytesToObjects);
+ JavaDStream<String> test2 = ZeroMQUtils.<String>createStream(
+ ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2());
+ JavaDStream<String> test3 = ZeroMQUtils.<String>createStream(
+ ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(),
+ SupervisorStrategy.defaultStrategy());
+ }
+}
diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..063529a9cb
--- /dev/null
+++ b/external/zeromq/src/test/resources/log4j.properties
@@ -0,0 +1,29 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file streaming/target/unit-tests.log
+log4j.rootCategory=INFO, file
+# log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
+
+# Ignore messages below warning level from Jetty, because it's a bit verbose
+log4j.logger.org.eclipse.jetty=WARN
+
diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
new file mode 100644
index 0000000000..4193b8a02f
--- /dev/null
+++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.zeromq
+
+import akka.actor.SupervisorStrategy
+import akka.util.ByteString
+import akka.zeromq.Subscribe
+
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.streaming.{StreamingContext, TestSuiteBase}
+
+class ZeroMQStreamSuite extends TestSuiteBase {
+
+ test("zeromq input stream") {
+ val ssc = new StreamingContext(master, framework, batchDuration)
+ val publishUrl = "abc"
+ val subscribe = new Subscribe(null.asInstanceOf[ByteString])
+ val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]]
+
+ // tests the API, does not actually test data receiving
+ val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects)
+ val test2 = ZeroMQUtils.createStream(
+ ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2)
+ val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects,
+ StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy)
+
+ // TODO: Actually test data receiving
+ }
+}
diff --git a/make-distribution.sh b/make-distribution.sh
index 32bbdb90a5..e6b5956d1e 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -31,10 +31,10 @@
#
# Recommended deploy/testing procedure (standalone mode):
# 1) Rsync / deploy the dist/ dir to one host
-# 2) cd to deploy dir; ./bin/start-master.sh
+# 2) cd to deploy dir; ./sbin/start-master.sh
# 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL.
-# 4) ./bin/start-slave.sh 1 <<spark:// URL>>
-# 5) MASTER="spark://my-master-ip:7077" ./spark-shell
+# 4) ./sbin/start-slave.sh 1 <<spark:// URL>>
+# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell
#
# Figure out where the Spark framework is installed
@@ -43,7 +43,17 @@ DISTDIR="$FWDIR/dist"
# Get version from SBT
export TERM=dumb # Prevents color codes in SBT output
-VERSION=$($FWDIR/sbt/sbt "show version" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/')
+
+VERSIONSTRING=$($FWDIR/sbt/sbt "show version")
+
+if [ $? == -1 ] ;then
+ echo -e "You need sbt installed and available on your path."
+ echo -e "Download sbt from http://www.scala-sbt.org/"
+ exit -1;
+fi
+
+VERSION=$(echo "${VERSIONSTRING}" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/')
+echo "Version is ${VERSION}"
# Initialize defaults
SPARK_HADOOP_VERSION=1.0.4
@@ -83,7 +93,9 @@ fi
# Build fat JAR
export SPARK_HADOOP_VERSION
export SPARK_YARN
-"$FWDIR/sbt/sbt" "assembly/assembly"
+cd $FWDIR
+
+"sbt/sbt" "assembly/assembly"
# Make directories
rm -rf "$DISTDIR"
@@ -98,10 +110,7 @@ mkdir "$DISTDIR"/conf
cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
cp -r "$FWDIR/bin" "$DISTDIR"
cp -r "$FWDIR/python" "$DISTDIR"
-cp "$FWDIR/spark-class" "$DISTDIR"
-cp "$FWDIR/spark-shell" "$DISTDIR"
-cp "$FWDIR/spark-executor" "$DISTDIR"
-cp "$FWDIR/pyspark" "$DISTDIR"
+cp -r "$FWDIR/sbin" "$DISTDIR"
if [ "$MAKE_TGZ" == "true" ]; then
diff --git a/mllib/pom.xml b/mllib/pom.xml
index f472082ad1..dda3900afe 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_2.10</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_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -48,12 +48,12 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -63,8 +63,8 @@
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
new file mode 100644
index 0000000000..2d8623392e
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -0,0 +1,262 @@
+/*
+ * 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.mllib.api.python
+import org.apache.spark.api.java.JavaRDD
+import org.apache.spark.mllib.regression._
+import org.apache.spark.mllib.classification._
+import org.apache.spark.mllib.clustering._
+import org.apache.spark.mllib.recommendation._
+import org.apache.spark.rdd.RDD
+import java.nio.ByteBuffer
+import java.nio.ByteOrder
+import java.nio.DoubleBuffer
+
+/**
+ * The Java stubs necessary for the Python mllib bindings.
+ */
+class PythonMLLibAPI extends Serializable {
+ private def deserializeDoubleVector(bytes: Array[Byte]): Array[Double] = {
+ val packetLength = bytes.length
+ if (packetLength < 16) {
+ throw new IllegalArgumentException("Byte array too short.")
+ }
+ val bb = ByteBuffer.wrap(bytes)
+ bb.order(ByteOrder.nativeOrder())
+ val magic = bb.getLong()
+ if (magic != 1) {
+ throw new IllegalArgumentException("Magic " + magic + " is wrong.")
+ }
+ val length = bb.getLong()
+ if (packetLength != 16 + 8 * length) {
+ throw new IllegalArgumentException("Length " + length + " is wrong.")
+ }
+ val db = bb.asDoubleBuffer()
+ val ans = new Array[Double](length.toInt)
+ db.get(ans)
+ return ans
+ }
+
+ private def serializeDoubleVector(doubles: Array[Double]): Array[Byte] = {
+ val len = doubles.length
+ val bytes = new Array[Byte](16 + 8 * len)
+ val bb = ByteBuffer.wrap(bytes)
+ bb.order(ByteOrder.nativeOrder())
+ bb.putLong(1)
+ bb.putLong(len)
+ val db = bb.asDoubleBuffer()
+ db.put(doubles)
+ return bytes
+ }
+
+ private def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = {
+ val packetLength = bytes.length
+ if (packetLength < 24) {
+ throw new IllegalArgumentException("Byte array too short.")
+ }
+ val bb = ByteBuffer.wrap(bytes)
+ bb.order(ByteOrder.nativeOrder())
+ val magic = bb.getLong()
+ if (magic != 2) {
+ throw new IllegalArgumentException("Magic " + magic + " is wrong.")
+ }
+ val rows = bb.getLong()
+ val cols = bb.getLong()
+ if (packetLength != 24 + 8 * rows * cols) {
+ throw new IllegalArgumentException("Size " + rows + "x" + cols + " is wrong.")
+ }
+ val db = bb.asDoubleBuffer()
+ val ans = new Array[Array[Double]](rows.toInt)
+ var i = 0
+ for (i <- 0 until rows.toInt) {
+ ans(i) = new Array[Double](cols.toInt)
+ db.get(ans(i))
+ }
+ return ans
+ }
+
+ private def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = {
+ val rows = doubles.length
+ var cols = 0
+ if (rows > 0) {
+ cols = doubles(0).length
+ }
+ val bytes = new Array[Byte](24 + 8 * rows * cols)
+ val bb = ByteBuffer.wrap(bytes)
+ bb.order(ByteOrder.nativeOrder())
+ bb.putLong(2)
+ bb.putLong(rows)
+ bb.putLong(cols)
+ val db = bb.asDoubleBuffer()
+ var i = 0
+ for (i <- 0 until rows) {
+ db.put(doubles(i))
+ }
+ return bytes
+ }
+
+ private def trainRegressionModel(trainFunc: (RDD[LabeledPoint], Array[Double]) => GeneralizedLinearModel,
+ dataBytesJRDD: JavaRDD[Array[Byte]], initialWeightsBA: Array[Byte]):
+ java.util.LinkedList[java.lang.Object] = {
+ val data = dataBytesJRDD.rdd.map(xBytes => {
+ val x = deserializeDoubleVector(xBytes)
+ LabeledPoint(x(0), x.slice(1, x.length))
+ })
+ val initialWeights = deserializeDoubleVector(initialWeightsBA)
+ val model = trainFunc(data, initialWeights)
+ val ret = new java.util.LinkedList[java.lang.Object]()
+ ret.add(serializeDoubleVector(model.weights))
+ ret.add(model.intercept: java.lang.Double)
+ return ret
+ }
+
+ /**
+ * Java stub for Python mllib LinearRegressionWithSGD.train()
+ */
+ def trainLinearRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]],
+ numIterations: Int, stepSize: Double, miniBatchFraction: Double,
+ initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+ return trainRegressionModel((data, initialWeights) =>
+ LinearRegressionWithSGD.train(data, numIterations, stepSize,
+ miniBatchFraction, initialWeights),
+ dataBytesJRDD, initialWeightsBA)
+ }
+
+ /**
+ * Java stub for Python mllib LassoWithSGD.train()
+ */
+ def trainLassoModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
+ stepSize: Double, regParam: Double, miniBatchFraction: Double,
+ initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+ return trainRegressionModel((data, initialWeights) =>
+ LassoWithSGD.train(data, numIterations, stepSize, regParam,
+ miniBatchFraction, initialWeights),
+ dataBytesJRDD, initialWeightsBA)
+ }
+
+ /**
+ * Java stub for Python mllib RidgeRegressionWithSGD.train()
+ */
+ def trainRidgeModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
+ stepSize: Double, regParam: Double, miniBatchFraction: Double,
+ initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+ return trainRegressionModel((data, initialWeights) =>
+ RidgeRegressionWithSGD.train(data, numIterations, stepSize, regParam,
+ miniBatchFraction, initialWeights),
+ dataBytesJRDD, initialWeightsBA)
+ }
+
+ /**
+ * Java stub for Python mllib SVMWithSGD.train()
+ */
+ def trainSVMModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
+ stepSize: Double, regParam: Double, miniBatchFraction: Double,
+ initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+ return trainRegressionModel((data, initialWeights) =>
+ SVMWithSGD.train(data, numIterations, stepSize, regParam,
+ miniBatchFraction, initialWeights),
+ dataBytesJRDD, initialWeightsBA)
+ }
+
+ /**
+ * Java stub for Python mllib LogisticRegressionWithSGD.train()
+ */
+ def trainLogisticRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]],
+ numIterations: Int, stepSize: Double, miniBatchFraction: Double,
+ initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
+ return trainRegressionModel((data, initialWeights) =>
+ LogisticRegressionWithSGD.train(data, numIterations, stepSize,
+ miniBatchFraction, initialWeights),
+ dataBytesJRDD, initialWeightsBA)
+ }
+
+ /**
+ * Java stub for Python mllib KMeans.train()
+ */
+ def trainKMeansModel(dataBytesJRDD: JavaRDD[Array[Byte]], k: Int,
+ maxIterations: Int, runs: Int, initializationMode: String):
+ java.util.List[java.lang.Object] = {
+ val data = dataBytesJRDD.rdd.map(xBytes => deserializeDoubleVector(xBytes))
+ val model = KMeans.train(data, k, maxIterations, runs, initializationMode)
+ val ret = new java.util.LinkedList[java.lang.Object]()
+ ret.add(serializeDoubleMatrix(model.clusterCenters))
+ return ret
+ }
+
+ /** Unpack a Rating object from an array of bytes */
+ private def unpackRating(ratingBytes: Array[Byte]): Rating = {
+ val bb = ByteBuffer.wrap(ratingBytes)
+ bb.order(ByteOrder.nativeOrder())
+ val user = bb.getInt()
+ val product = bb.getInt()
+ val rating = bb.getDouble()
+ return new Rating(user, product, rating)
+ }
+
+ /** Unpack a tuple of Ints from an array of bytes */
+ private[spark] def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = {
+ val bb = ByteBuffer.wrap(tupleBytes)
+ bb.order(ByteOrder.nativeOrder())
+ val v1 = bb.getInt()
+ val v2 = bb.getInt()
+ (v1, v2)
+ }
+
+ /**
+ * Serialize a Rating object into an array of bytes.
+ * It can be deserialized using RatingDeserializer().
+ *
+ * @param rate
+ * @return
+ */
+ private[spark] def serializeRating(rate: Rating): Array[Byte] = {
+ val len = 3
+ val bytes = new Array[Byte](4 + 8 * len)
+ val bb = ByteBuffer.wrap(bytes)
+ bb.order(ByteOrder.nativeOrder())
+ bb.putInt(len)
+ val db = bb.asDoubleBuffer()
+ db.put(rate.user.toDouble)
+ db.put(rate.product.toDouble)
+ db.put(rate.rating)
+ bytes
+ }
+
+ /**
+ * Java stub for Python mllib ALS.train(). This stub returns a handle
+ * to the Java object instead of the content of the Java object. Extra care
+ * needs to be taken in the Python code to ensure it gets freed on exit; see
+ * the Py4J documentation.
+ */
+ def trainALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int,
+ iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = {
+ val ratings = ratingsBytesJRDD.rdd.map(unpackRating)
+ return ALS.train(ratings, rank, iterations, lambda, blocks)
+ }
+
+ /**
+ * Java stub for Python mllib ALS.trainImplicit(). This stub returns a
+ * handle to the Java object instead of the content of the Java object.
+ * Extra care needs to be taken in the Python code to ensure it gets freed on
+ * exit; see the Py4J documentation.
+ */
+ def trainImplicitALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int,
+ iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = {
+ val ratings = ratingsBytesJRDD.rdd.map(unpackRating)
+ return ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha)
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
new file mode 100644
index 0000000000..524300d6ae
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
@@ -0,0 +1,119 @@
+/*
+ * 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.mllib.classification
+
+import scala.collection.mutable
+
+import org.jblas.DoubleMatrix
+
+import org.apache.spark.Logging
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.rdd.RDD
+
+/**
+ * Model for Naive Bayes Classifiers.
+ *
+ * @param pi Log of class priors, whose dimension is C.
+ * @param theta Log of class conditional probabilities, whose dimension is CXD.
+ */
+class NaiveBayesModel(pi: Array[Double], theta: Array[Array[Double]])
+ extends ClassificationModel with Serializable {
+
+ // Create a column vector that can be used for predictions
+ private val _pi = new DoubleMatrix(pi.length, 1, pi: _*)
+ private val _theta = new DoubleMatrix(theta)
+
+ def predict(testData: RDD[Array[Double]]): RDD[Double] = testData.map(predict)
+
+ def predict(testData: Array[Double]): Double = {
+ val dataMatrix = new DoubleMatrix(testData.length, 1, testData: _*)
+ val result = _pi.add(_theta.mmul(dataMatrix))
+ result.argmax()
+ }
+}
+
+/**
+ * Trains a Naive Bayes model given an RDD of `(label, features)` pairs.
+ *
+ * @param lambda The smooth parameter
+ */
+class NaiveBayes private (val lambda: Double = 1.0)
+ extends Serializable with Logging {
+
+ /**
+ * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries.
+ *
+ * @param data RDD of (label, array of features) pairs.
+ */
+ def run(data: RDD[LabeledPoint]) = {
+ // Aggregates all sample points to driver side to get sample count and summed feature vector
+ // for each label. The shape of `zeroCombiner` & `aggregated` is:
+ //
+ // label: Int -> (count: Int, featuresSum: DoubleMatrix)
+ val zeroCombiner = mutable.Map.empty[Int, (Int, DoubleMatrix)]
+ val aggregated = data.aggregate(zeroCombiner)({ (combiner, point) =>
+ point match {
+ case LabeledPoint(label, features) =>
+ val (count, featuresSum) = combiner.getOrElse(label.toInt, (0, DoubleMatrix.zeros(1)))
+ val fs = new DoubleMatrix(features.length, 1, features: _*)
+ combiner += label.toInt -> (count + 1, featuresSum.addi(fs))
+ }
+ }, { (lhs, rhs) =>
+ for ((label, (c, fs)) <- rhs) {
+ val (count, featuresSum) = lhs.getOrElse(label, (0, DoubleMatrix.zeros(1)))
+ lhs(label) = (count + c, featuresSum.addi(fs))
+ }
+ lhs
+ })
+
+ // Kinds of label
+ val C = aggregated.size
+ // Total sample count
+ val N = aggregated.values.map(_._1).sum
+
+ val pi = new Array[Double](C)
+ val theta = new Array[Array[Double]](C)
+ val piLogDenom = math.log(N + C * lambda)
+
+ for ((label, (count, fs)) <- aggregated) {
+ val thetaLogDenom = math.log(fs.sum() + fs.length * lambda)
+ pi(label) = math.log(count + lambda) - piLogDenom
+ theta(label) = fs.toArray.map(f => math.log(f + lambda) - thetaLogDenom)
+ }
+
+ new NaiveBayesModel(pi, theta)
+ }
+}
+
+object NaiveBayes {
+ /**
+ * Trains a Naive Bayes model given an RDD of `(label, features)` pairs.
+ *
+ * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of
+ * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for
+ * document classification. By making every vector a 0-1 vector. it can also be used as
+ * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]).
+ *
+ * @param input RDD of `(label, array of features)` pairs. Every vector should be a frequency
+ * vector or a count vector.
+ * @param lambda The smooth parameter
+ */
+ def train(input: RDD[LabeledPoint], lambda: Double = 1.0): NaiveBayesModel = {
+ new NaiveBayes(lambda).run(input)
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
index 749e7364f4..c590492e7a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
@@ -50,8 +50,8 @@ class LogisticGradient extends Gradient {
val gradient = data.mul(gradientMultiplier)
val loss =
- if (margin > 0) {
- math.log(1 + math.exp(0 - margin))
+ if (label > 0) {
+ math.log(1 + math.exp(margin))
} else {
math.log(1 + math.exp(margin)) - margin
}
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 36853acab5..8b27ecf82c 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
@@ -578,14 +578,13 @@ object ALS {
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")
+ sc.conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
+ sc.conf.set("spark.kryo.registrator", classOf[ALSRegistrator].getName)
+ sc.conf.set("spark.kryo.referenceTracking", "false")
+ sc.conf.set("spark.kryoserializer.buffer.mb", "8")
+ sc.conf.set("spark.locality.wait", "10000")
+
val ratings = sc.textFile(ratingsFile).map { line =>
val fields = line.split(',')
Rating(fields(0).toInt, fields(1).toInt, fields(2).toDouble)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
index af43d89c70..443fc5de5b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
@@ -19,8 +19,11 @@ package org.apache.spark.mllib.recommendation
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.api.python.PythonMLLibAPI
import org.jblas._
+import org.apache.spark.api.java.JavaRDD
+
/**
* Model representing the result of matrix factorization.
@@ -44,6 +47,39 @@ class MatrixFactorizationModel(
userVector.dot(productVector)
}
- // TODO: Figure out what good bulk prediction methods would look like.
+ /**
+ * Predict the rating of many users for many products.
+ * The output RDD has an element per each element in the input RDD (including all duplicates)
+ * unless a user or product is missing in the training set.
+ *
+ * @param usersProducts RDD of (user, product) pairs.
+ * @return RDD of Ratings.
+ */
+ def predict(usersProducts: RDD[(Int, Int)]): RDD[Rating] = {
+ val users = userFeatures.join(usersProducts).map{
+ case (user, (uFeatures, product)) => (product, (user, uFeatures))
+ }
+ users.join(productFeatures).map {
+ case (product, ((user, uFeatures), pFeatures)) =>
+ val userVector = new DoubleMatrix(uFeatures)
+ val productVector = new DoubleMatrix(pFeatures)
+ Rating(user, product, userVector.dot(productVector))
+ }
+ }
+
+ /**
+ * Predict the rating of many users for many products.
+ * This is a Java stub for python predictAll()
+ *
+ * @param usersProductsJRDD A JavaRDD with serialized tuples (user, product)
+ * @return JavaRDD of serialized Rating objects.
+ */
+ def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = {
+ val pythonAPI = new PythonMLLibAPI()
+ val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes))
+ predict(usersProducts).map(rate => pythonAPI.serializeRating(rate))
+ }
+
+ // TODO: Figure out what other good bulk prediction methods would look like.
// Probably want a way to get the top users for a product or vice-versa.
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index 5aec867257..d5f3f6b8db 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
@@ -83,7 +83,7 @@ object MFDataGenerator{
scala.math.round(.99 * m * n)).toInt
val rand = new Random()
val mn = m * n
- val shuffled = rand.shuffle(1 to mn toIterable)
+ val shuffled = rand.shuffle(1 to mn toList)
val omega = shuffled.slice(0, sampSize)
val ordered = omega.sortWith(_ < _).toArray
diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
index 32d3934ac1..33b99f4bd3 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java
@@ -77,7 +77,7 @@ public class JavaKMeansSuite implements Serializable {
@Test
public void runKMeansUsingStaticMethods() {
- List<double[]> points = new ArrayList();
+ List<double[]> points = new ArrayList<double[]>();
points.add(new double[]{1.0, 2.0, 6.0});
points.add(new double[]{1.0, 3.0, 0.0});
points.add(new double[]{1.0, 4.0, 6.0});
@@ -94,7 +94,7 @@ public class JavaKMeansSuite implements Serializable {
@Test
public void runKMeansUsingConstructor() {
- List<double[]> points = new ArrayList();
+ List<double[]> points = new ArrayList<double[]>();
points.add(new double[]{1.0, 2.0, 6.0});
points.add(new double[]{1.0, 3.0, 0.0});
points.add(new double[]{1.0, 4.0, 6.0});
diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java
index eafee060cd..b40f552e0d 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
@@ -21,8 +21,6 @@ import java.io.Serializable;
import java.util.List;
import java.lang.Math;
-import scala.Tuple2;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
new file mode 100644
index 0000000000..b615f76e66
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
@@ -0,0 +1,108 @@
+/*
+ * 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.mllib.classification
+
+import scala.util.Random
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.FunSuite
+
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.SparkContext
+
+object NaiveBayesSuite {
+
+ private def calcLabel(p: Double, pi: Array[Double]): Int = {
+ var sum = 0.0
+ for (j <- 0 until pi.length) {
+ sum += pi(j)
+ if (p < sum) return j
+ }
+ -1
+ }
+
+ // Generate input of the form Y = (theta * x).argmax()
+ def generateNaiveBayesInput(
+ pi: Array[Double], // 1XC
+ theta: Array[Array[Double]], // CXD
+ nPoints: Int,
+ seed: Int): Seq[LabeledPoint] = {
+ val D = theta(0).length
+ val rnd = new Random(seed)
+
+ val _pi = pi.map(math.pow(math.E, _))
+ val _theta = theta.map(row => row.map(math.pow(math.E, _)))
+
+ for (i <- 0 until nPoints) yield {
+ val y = calcLabel(rnd.nextDouble(), _pi)
+ val xi = Array.tabulate[Double](D) { j =>
+ if (rnd.nextDouble() < _theta(y)(j)) 1 else 0
+ }
+
+ LabeledPoint(y, xi)
+ }
+ }
+}
+
+class NaiveBayesSuite extends FunSuite with BeforeAndAfterAll {
+ @transient private var sc: SparkContext = _
+
+ override def beforeAll() {
+ sc = new SparkContext("local", "test")
+ }
+
+ override def afterAll() {
+ sc.stop()
+ System.clearProperty("spark.driver.port")
+ }
+
+ def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) {
+ val numOfPredictions = predictions.zip(input).count {
+ case (prediction, expected) =>
+ prediction != expected.label
+ }
+ // At least 80% of the predictions should be on.
+ assert(numOfPredictions < input.length / 5)
+ }
+
+ test("Naive Bayes") {
+ val nPoints = 10000
+
+ val pi = Array(0.5, 0.3, 0.2).map(math.log)
+ val theta = Array(
+ Array(0.91, 0.03, 0.03, 0.03), // label 0
+ Array(0.03, 0.91, 0.03, 0.03), // label 1
+ Array(0.03, 0.03, 0.91, 0.03) // label 2
+ ).map(_.map(math.log))
+
+ val testData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 42)
+ val testRDD = sc.parallelize(testData, 2)
+ testRDD.cache()
+
+ val model = NaiveBayes.train(testRDD)
+
+ val validationData = NaiveBayesSuite.generateNaiveBayesInput(pi, theta, nPoints, 17)
+ val validationRDD = sc.parallelize(validationData, 2)
+
+ // Test prediction on RDD.
+ validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData)
+
+ // Test prediction on Array.
+ validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
+ }
+}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala
new file mode 100644
index 0000000000..a6028a1e98
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala
@@ -0,0 +1,116 @@
+/*
+ * 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.mllib.optimization
+
+import scala.util.Random
+import scala.collection.JavaConversions._
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.FunSuite
+import org.scalatest.matchers.ShouldMatchers
+
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.regression._
+
+object GradientDescentSuite {
+
+ def generateLogisticInputAsList(
+ offset: Double,
+ scale: Double,
+ nPoints: Int,
+ seed: Int): java.util.List[LabeledPoint] = {
+ seqAsJavaList(generateGDInput(offset, scale, nPoints, seed))
+ }
+
+ // Generate input of the form Y = logistic(offset + scale * X)
+ def generateGDInput(
+ offset: Double,
+ scale: Double,
+ nPoints: Int,
+ seed: Int): Seq[LabeledPoint] = {
+ val rnd = new Random(seed)
+ val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian())
+
+ val unifRand = new scala.util.Random(45)
+ val rLogis = (0 until nPoints).map { i =>
+ val u = unifRand.nextDouble()
+ math.log(u) - math.log(1.0-u)
+ }
+
+ val y: Seq[Int] = (0 until nPoints).map { i =>
+ val yVal = offset + scale * x1(i) + rLogis(i)
+ if (yVal > 0) 1 else 0
+ }
+
+ val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i))))
+ testData
+ }
+}
+
+class GradientDescentSuite extends FunSuite with BeforeAndAfterAll with ShouldMatchers {
+ @transient private var sc: SparkContext = _
+
+ override def beforeAll() {
+ sc = new SparkContext("local", "test")
+ }
+
+ override def afterAll() {
+ sc.stop()
+ System.clearProperty("spark.driver.port")
+ }
+
+ test("Assert the loss is decreasing.") {
+ val nPoints = 10000
+ val A = 2.0
+ val B = -1.5
+
+ val initialB = -1.0
+ val initialWeights = Array(initialB)
+
+ val gradient = new LogisticGradient()
+ val updater = new SimpleUpdater()
+ val stepSize = 1.0
+ val numIterations = 10
+ val regParam = 0
+ val miniBatchFrac = 1.0
+
+ // Add a extra variable consisting of all 1.0's for the intercept.
+ val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42)
+ val data = testData.map { case LabeledPoint(label, features) =>
+ label -> Array(1.0, features: _*)
+ }
+
+ val dataRDD = sc.parallelize(data, 2).cache()
+ val initialWeightsWithIntercept = Array(1.0, initialWeights: _*)
+
+ val (_, loss) = GradientDescent.runMiniBatchSGD(
+ dataRDD,
+ gradient,
+ updater,
+ stepSize,
+ numIterations,
+ regParam,
+ miniBatchFrac,
+ initialWeightsWithIntercept)
+
+ assert(loss.last - loss.head < 0, "loss isn't decreasing.")
+
+ val lossDiff = loss.init.zip(loss.tail).map { case (lhs, rhs) => lhs - rhs }
+ assert(lossDiff.count(_ > 0).toDouble / lossDiff.size > 0.8)
+ }
+}
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 fafc5ec5f2..e683a90f57 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
@@ -90,18 +90,34 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
testALS(50, 100, 1, 15, 0.7, 0.3)
}
+ test("rank-1 matrices bulk") {
+ testALS(50, 100, 1, 15, 0.7, 0.3, false, true)
+ }
+
test("rank-2 matrices") {
testALS(100, 200, 2, 15, 0.7, 0.3)
}
+ test("rank-2 matrices bulk") {
+ testALS(100, 200, 2, 15, 0.7, 0.3, false, true)
+ }
+
test("rank-1 matrices implicit") {
testALS(80, 160, 1, 15, 0.7, 0.4, true)
}
+ test("rank-1 matrices implicit bulk") {
+ testALS(80, 160, 1, 15, 0.7, 0.4, true, true)
+ }
+
test("rank-2 matrices implicit") {
testALS(100, 200, 2, 15, 0.7, 0.4, true)
}
+ test("rank-2 matrices implicit bulk") {
+ testALS(100, 200, 2, 15, 0.7, 0.4, true, true)
+ }
+
/**
* Test if we can correctly factorize R = U * P where U and P are of known rank.
*
@@ -111,9 +127,12 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
* @param iterations number of iterations to run
* @param samplingRate what fraction of the user-product pairs are known
* @param matchThreshold max difference allowed to consider a predicted rating correct
+ * @param implicitPrefs flag to test implicit feedback
+ * @param bulkPredict flag to test bulk prediciton
*/
def testALS(users: Int, products: Int, features: Int, iterations: Int,
- samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false)
+ samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false,
+ bulkPredict: Boolean = false)
{
val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products,
features, samplingRate, implicitPrefs)
@@ -130,7 +149,17 @@ class ALSSuite extends FunSuite with BeforeAndAfterAll {
for ((p, vec) <- model.productFeatures.collect(); i <- 0 until features) {
predictedP.put(p, i, vec(i))
}
- val predictedRatings = predictedU.mmul(predictedP.transpose)
+ val predictedRatings = bulkPredict match {
+ case false => predictedU.mmul(predictedP.transpose)
+ case true =>
+ val allRatings = new DoubleMatrix(users, products)
+ val usersProducts = for (u <- 0 until users; p <- 0 until products) yield (u, p)
+ val userProductsRDD = sc.parallelize(usersProducts)
+ model.predict(userProductsRDD).collect().foreach { elem =>
+ allRatings.put(elem.user, elem.product, elem.rating)
+ }
+ allRatings
+ }
if (!implicitPrefs) {
for (u <- 0 until users; p <- 0 until products) {
diff --git a/pom.xml b/pom.xml
index a9eccd69cc..ba19c51391 100644
--- a/pom.xml
+++ b/pom.xml
@@ -88,25 +88,34 @@
<module>core</module>
<module>bagel</module>
<module>graph</module>
- <module>examples</module>
<module>mllib</module>
<module>tools</module>
<module>streaming</module>
<module>repl</module>
<module>assembly</module>
+ <module>external/twitter</module>
+ <module>external/kafka</module>
+ <module>external/flume</module>
+ <module>external/zeromq</module>
+ <module>external/mqtt</module>
+ <module>examples</module>
</modules>
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
- <java.version>1.5</java.version>
- <scala.version>2.9.3</scala.version>
+ <java.version>1.6</java.version>
+
+ <scala.version>2.10.3</scala.version>
+ <scala.binary.version>2.10</scala.binary.version>
<mesos.version>0.13.0</mesos.version>
- <akka.version>2.0.5</akka.version>
+ <akka.group>org.spark-project.akka</akka.group>
+ <akka.version>2.2.3-shaded-protobuf</akka.version>
<slf4j.version>1.7.2</slf4j.version>
<log4j.version>1.2.17</log4j.version>
<hadoop.version>1.0.4</hadoop.version>
+ <protobuf.version>2.4.1</protobuf.version>
<yarn.version>0.23.7</yarn.version>
<hbase.version>0.94.6</hbase.version>
@@ -116,9 +125,9 @@
<repositories>
<repository>
- <id>jboss-repo</id>
- <name>JBoss Repository</name>
- <url>http://repository.jboss.org/nexus/content/repositories/releases/</url>
+ <id>maven-repo</id> <!-- This should be at top, it makes maven try the central repo first and then others and hence faster dep resolution -->
+ <name>Maven Repository</name>
+ <url>http://repo.maven.apache.org/maven2</url>
<releases>
<enabled>true</enabled>
</releases>
@@ -127,20 +136,9 @@
</snapshots>
</repository>
<repository>
- <id>cloudera-repo</id>
- <name>Cloudera Repository</name>
- <url>https://repository.cloudera.com/artifactory/cloudera-repos/</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>false</enabled>
- </snapshots>
- </repository>
- <repository>
- <id>akka-repo</id>
- <name>Akka Repository</name>
- <url>http://repo.akka.io/releases/</url>
+ <id>jboss-repo</id>
+ <name>JBoss Repository</name>
+ <url>http://repository.jboss.org/nexus/content/repositories/releases</url>
<releases>
<enabled>true</enabled>
</releases>
@@ -149,52 +147,11 @@
</snapshots>
</repository>
<repository>
- <id>mqtt-repo</id>
- <name>MQTT Repository</name>
- <url>https://repo.eclipse.org/content/repositories/paho-releases/</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>false</enabled>
- </snapshots>
+ <id>cloudera-repo</id>
+ <name>Cloudera Repository</name>
+ <url>https://repository.cloudera.com/artifactory/cloudera-repos</url>
</repository>
</repositories>
- <pluginRepositories>
- <pluginRepository>
- <id>oss-sonatype-releases</id>
- <name>OSS Sonatype</name>
- <url>https://oss.sonatype.org/content/repositories/releases</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>false</enabled>
- </snapshots>
- </pluginRepository>
- <pluginRepository>
- <id>oss-sonatype-snapshots</id>
- <name>OSS Sonatype</name>
- <url>https://oss.sonatype.org/content/repositories/snapshots</url>
- <releases>
- <enabled>false</enabled>
- </releases>
- <snapshots>
- <enabled>true</enabled>
- </snapshots>
- </pluginRepository>
- <pluginRepository>
- <id>oss-sonatype</id>
- <name>OSS Sonatype</name>
- <url>https://oss.sonatype.org/content/groups/public</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>true</enabled>
- </snapshots>
- </pluginRepository>
- </pluginRepositories>
<dependencyManagement>
<dependencies>
@@ -231,7 +188,7 @@
<dependency>
<groupId>com.ning</groupId>
<artifactId>compress-lzf</artifactId>
- <version>0.8.4</version>
+ <version>1.0.0</version>
</dependency>
<dependency>
<groupId>org.xerial.snappy</groupId>
@@ -244,13 +201,23 @@
<version>4.0</version>
</dependency>
<dependency>
+ <groupId>com.clearspring.analytics</groupId>
+ <artifactId>stream</artifactId>
+ <version>2.4.0</version>
+ </dependency>
+ <!-- In theory we need not directly depend on protobuf since Spark does not directly
+ use it. However, when building with Hadoop/YARN 2.2 Maven doesn't correctly bump
+ the protobuf version up from the one Mesos gives. For now we include this variable
+ to explicitly bump the version when building with YARN. It would be nice to figure
+ out why Maven can't resolve this correctly (like SBT does). -->
+ <dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
- <version>2.4.1</version>
+ <version>${protobuf.version}</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
- <artifactId>chill_2.9.3</artifactId>
+ <artifactId>chill_${scala.binary.version}</artifactId>
<version>0.3.1</version>
</dependency>
<dependency>
@@ -259,8 +226,8 @@
<version>0.3.1</version>
</dependency>
<dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-actor</artifactId>
+ <groupId>${akka.group}</groupId>
+ <artifactId>akka-actor_${scala.binary.version}</artifactId>
<version>${akka.version}</version>
<exclusions>
<exclusion>
@@ -270,8 +237,8 @@
</exclusions>
</dependency>
<dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-remote</artifactId>
+ <groupId>${akka.group}</groupId>
+ <artifactId>akka-remote_${scala.binary.version}</artifactId>
<version>${akka.version}</version>
<exclusions>
<exclusion>
@@ -281,8 +248,8 @@
</exclusions>
</dependency>
<dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-slf4j</artifactId>
+ <groupId>${akka.group}</groupId>
+ <artifactId>akka-slf4j_${scala.binary.version}</artifactId>
<version>${akka.version}</version>
<exclusions>
<exclusion>
@@ -302,11 +269,6 @@
<version>1.2.0</version>
</dependency>
<dependency>
- <groupId>com.github.scala-incubator.io</groupId>
- <artifactId>scala-io-file_2.9.2</artifactId>
- <version>0.4.1</version>
- </dependency>
- <dependency>
<groupId>org.apache.mesos</groupId>
<artifactId>mesos</artifactId>
<version>${mesos.version}</version>
@@ -314,7 +276,7 @@
<dependency>
<groupId>io.netty</groupId>
<artifactId>netty-all</artifactId>
- <version>4.0.0.Beta2</version>
+ <version>4.0.13.Final</version>
</dependency>
<dependency>
<groupId>org.apache.derby</groupId>
@@ -324,8 +286,14 @@
</dependency>
<dependency>
<groupId>net.liftweb</groupId>
- <artifactId>lift-json_2.9.2</artifactId>
- <version>2.5</version>
+ <artifactId>lift-json_${scala.binary.version}</artifactId>
+ <version>2.5.1</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.scala-lang</groupId>
+ <artifactId>scalap</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
<groupId>com.codahale.metrics</groupId>
@@ -368,24 +336,22 @@
<version>${scala.version}</version>
</dependency>
<dependency>
- <groupId>org.scala-lang</groupId>
- <artifactId>scalap</artifactId>
- <version>${scala.version}</version>
- </dependency>
-
- <dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<version>${log4j.version}</version>
</dependency>
-
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<version>1.9.1</version>
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>2.4</version>
+ </dependency>
+ <dependency>
<groupId>org.easymock</groupId>
<artifactId>easymock</artifactId>
<version>3.1</version>
@@ -399,7 +365,7 @@
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
<version>1.10.0</version>
<scope>test</scope>
</dependency>
@@ -494,6 +460,7 @@
</exclusion>
</exclusions>
</dependency>
+
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-client</artifactId>
@@ -532,6 +499,10 @@
<groupId>org.jboss.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
</exclusions>
</dependency>
</dependencies>
@@ -740,7 +711,7 @@
<profiles>
<profile>
- <id>hadoop2-yarn</id>
+ <id>yarn-alpha</id>
<properties>
<hadoop.major.version>2</hadoop.major.version>
<!-- 0.23.* is same as 2.0.* - except hardened to run production jobs -->
@@ -752,25 +723,21 @@
<module>yarn</module>
</modules>
- <repositories>
- <repository>
- <id>maven-root</id>
- <name>Maven root repository</name>
- <url>http://repo1.maven.org/maven2/</url>
- <releases>
- <enabled>true</enabled>
- </releases>
- <snapshots>
- <enabled>false</enabled>
- </snapshots>
- </repository>
- </repositories>
+ </profile>
+
+ <profile>
+ <id>yarn</id>
+ <properties>
+ <hadoop.major.version>2</hadoop.major.version>
+ <hadoop.version>2.2.0</hadoop.version>
+ <protobuf.version>2.5.0</protobuf.version>
+ </properties>
+ <modules>
+ <module>yarn</module>
+ </modules>
- <dependencyManagement>
- <dependencies>
- </dependencies>
- </dependencyManagement>
</profile>
+
<profile>
<id>repl-bin</id>
<activation>
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 47906ada74..c2cd6fb45a 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -20,6 +20,7 @@ import sbt.Classpaths.publishTask
import Keys._
import sbtassembly.Plugin._
import AssemblyKeys._
+import scala.util.Properties
// For Sonatype publishing
//import com.jsuereth.pgp.sbtplugin.PgpKeys._
@@ -28,14 +29,19 @@ object SparkBuild extends Build {
// "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set
// through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN.
val DEFAULT_HADOOP_VERSION = "1.0.4"
+
+ // Whether the Hadoop version to build against is 2.2.x, or a variant of it. This can be set
+ // through the SPARK_IS_NEW_HADOOP environment variable.
+ val DEFAULT_IS_NEW_HADOOP = false
+
val DEFAULT_YARN = false
// 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"
+ val SCALAC_JVM_VERSION = "jvm-1.6"
+ val JAVAC_JVM_VERSION = "1.6"
lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects: _*)
@@ -44,21 +50,16 @@ object SparkBuild extends Build {
lazy val repl = Project("repl", file("repl"), settings = replSettings)
.dependsOn(core, graph, bagel, mllib)
- lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
- .dependsOn(core, mllib, graph, bagel, streaming)
-
lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming)
lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core)
- lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core)
+ lazy val graph = Project("graph", file("graph"), settings = graphSettings) dependsOn(core)
lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core)
lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core)
- lazy val yarn = Project("yarn", file("yarn"), settings = yarnSettings) dependsOn(core)
-
lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings)
.dependsOn(core, graph, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*)
@@ -67,27 +68,60 @@ object SparkBuild extends Build {
// A configuration to set an alternative publishLocalConfiguration
lazy val MavenCompile = config("m2r") extend(Compile)
lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
+ val sparkHome = System.getProperty("user.dir")
// Allows build configuration to be set through environment variables
- lazy val hadoopVersion = scala.util.Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
- lazy val isYarnEnabled = scala.util.Properties.envOrNone("SPARK_YARN") match {
+ lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
+ lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match {
+ case None => {
+ val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined
+ (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP)
+ }
+ case Some(v) => v.toBoolean
+ }
+
+ lazy val isYarnEnabled = Properties.envOrNone("SPARK_YARN") match {
case None => DEFAULT_YARN
case Some(v) => v.toBoolean
}
// Conditionally include the yarn sub-project
- lazy val maybeYarn = if(isYarnEnabled) Seq[ClasspathDependency](yarn) else Seq[ClasspathDependency]()
- lazy val maybeYarnRef = if(isYarnEnabled) Seq[ProjectReference](yarn) else Seq[ProjectReference]()
+ lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core)
+ lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core)
+
+ lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]()
+ lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]()
+
+ lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings)
+ .dependsOn(streaming % "compile->compile;test->test")
+
+ lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings)
+ .dependsOn(streaming % "compile->compile;test->test")
+
+ lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings)
+ .dependsOn(streaming % "compile->compile;test->test")
+
+ lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings)
+ .dependsOn(streaming % "compile->compile;test->test")
+
+ lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings)
+ .dependsOn(streaming % "compile->compile;test->test")
+
+ lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
+ lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt)
+
+ lazy val examples = Project("examples", file("examples"), settings = examplesSettings)
+ .dependsOn(core, mllib, graph, bagel, streaming, externalTwitter) dependsOn(allExternal: _*)
// Everything except assembly, tools and examples belong to packageProjects
lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graph) ++ maybeYarnRef
- lazy val allProjects = packageProjects ++ Seq[ProjectReference](examples, tools, assemblyProj)
+ lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj)
def sharedSettings = Defaults.defaultSettings ++ Seq(
- organization := "org.apache.spark",
- version := "0.9.0-incubating-SNAPSHOT",
- scalaVersion := "2.9.3",
+ organization := "org.apache.spark",
+ version := "0.9.0-incubating-SNAPSHOT",
+ scalaVersion := "2.10.3",
scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation",
"-target:" + SCALAC_JVM_VERSION),
javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION),
@@ -99,7 +133,11 @@ object SparkBuild extends Build {
// Fork new JVMs for tests and set Java options for those
fork := true,
+ javaOptions in Test += "-Dspark.home=" + sparkHome,
+ javaOptions in Test += "-Dspark.testing=1",
javaOptions += "-Xmx3g",
+ // Show full stack trace and duration in test cases.
+ testOptions in Test += Tests.Argument("-oDF"),
// 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),
@@ -107,13 +145,7 @@ object SparkBuild extends Build {
// also check the local Maven repository ~/.m2
resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))),
- // Shared between both core and streaming.
- resolvers ++= Seq("Akka Repository" at "http://repo.akka.io/releases/"),
-
- // Shared between both examples and streaming.
- resolvers ++= Seq("Mqtt Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/"),
-
- // For Sonatype publishing
+ // For Sonatype publishing
resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"),
@@ -155,7 +187,7 @@ object SparkBuild extends Build {
</issueManagement>
),
-/*
+ /*
publishTo <<= version { (v: String) =>
val nexus = "https://oss.sonatype.org/"
if (v.trim.endsWith("SNAPSHOT"))
@@ -164,17 +196,22 @@ object SparkBuild extends Build {
Some("sonatype-staging" at nexus + "service/local/staging/deploy/maven2")
},
-*/
-
+ */
libraryDependencies ++= Seq(
- "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106",
- "org.scalatest" %% "scalatest" % "1.9.1" % "test",
- "org.scalacheck" %% "scalacheck" % "1.10.0" % "test",
- "com.novocode" % "junit-interface" % "0.9" % "test",
- "org.easymock" % "easymock" % "3.1" % "test",
- "org.mockito" % "mockito-all" % "1.8.5" % "test"
+ "io.netty" % "netty-all" % "4.0.13.Final",
+ "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106",
+ /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */
+ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"),
+ "org.scalatest" %% "scalatest" % "1.9.1" % "test",
+ "org.scalacheck" %% "scalacheck" % "1.10.0" % "test",
+ "com.novocode" % "junit-interface" % "0.9" % "test",
+ "org.easymock" % "easymock" % "3.1" % "test",
+ "org.mockito" % "mockito-all" % "1.8.5" % "test",
+ "commons-io" % "commons-io" % "2.4" % "test"
),
+
+ parallelExecution := true,
/* Workaround for issue #206 (fixed after SBT 0.11.0) */
watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task,
const(std.TaskExtra.constant(Nil)), aggregate = true, includeRoot = true) apply { _.join.map(_.flatten) },
@@ -199,62 +236,62 @@ object SparkBuild extends Build {
def coreSettings = sharedSettings ++ Seq(
name := "spark-core",
resolvers ++= Seq(
- "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
- "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
+ "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
+ "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
),
libraryDependencies ++= Seq(
- "com.google.guava" % "guava" % "14.0.1",
- "com.google.code.findbugs" % "jsr305" % "1.3.9",
- "log4j" % "log4j" % "1.2.17",
- "org.slf4j" % "slf4j-api" % slf4jVersion,
- "org.slf4j" % "slf4j-log4j12" % slf4jVersion,
- "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407
- "com.ning" % "compress-lzf" % "0.8.4",
- "org.xerial.snappy" % "snappy-java" % "1.0.5",
- "org.ow2.asm" % "asm" % "4.0",
- "com.google.protobuf" % "protobuf-java" % "2.4.1",
- "com.typesafe.akka" % "akka-actor" % "2.0.5" excludeAll(excludeNetty),
- "com.typesafe.akka" % "akka-remote" % "2.0.5" excludeAll(excludeNetty),
- "com.typesafe.akka" % "akka-slf4j" % "2.0.5" excludeAll(excludeNetty),
- "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.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, 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.codahale.metrics" % "metrics-graphite" % "3.0.0",
- "com.twitter" % "chill_2.9.3" % "0.3.1",
- "com.twitter" % "chill-java" % "0.3.1"
- )
+ "com.google.guava" % "guava" % "14.0.1",
+ "com.google.code.findbugs" % "jsr305" % "1.3.9",
+ "log4j" % "log4j" % "1.2.17",
+ "org.slf4j" % "slf4j-api" % slf4jVersion,
+ "org.slf4j" % "slf4j-log4j12" % slf4jVersion,
+ "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407
+ "com.ning" % "compress-lzf" % "1.0.0",
+ "org.xerial.snappy" % "snappy-java" % "1.0.5",
+ "org.ow2.asm" % "asm" % "4.0",
+ "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty),
+ "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty),
+ "net.liftweb" %% "lift-json" % "2.5.1" excludeAll(excludeNetty),
+ "it.unimi.dsi" % "fastutil" % "6.4.4",
+ "colt" % "colt" % "1.2.0",
+ "org.apache.mesos" % "mesos" % "0.13.0",
+ "net.java.dev.jets3t" % "jets3t" % "0.7.1",
+ "org.apache.derby" % "derby" % "10.4.2.0" % "test",
+ "org.apache.hadoop" % "hadoop-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib),
+ "org.apache.avro" % "avro" % "1.7.4",
+ "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty),
+ "org.apache.zookeeper" % "zookeeper" % "3.4.5" excludeAll(excludeNetty),
+ "com.codahale.metrics" % "metrics-core" % "3.0.0",
+ "com.codahale.metrics" % "metrics-jvm" % "3.0.0",
+ "com.codahale.metrics" % "metrics-json" % "3.0.0",
+ "com.codahale.metrics" % "metrics-ganglia" % "3.0.0",
+ "com.codahale.metrics" % "metrics-graphite" % "3.0.0",
+ "com.twitter" %% "chill" % "0.3.1",
+ "com.twitter" % "chill-java" % "0.3.1",
+ "com.typesafe" % "config" % "1.0.2",
+ "com.clearspring.analytics" % "stream" % "2.5.1"
+ )
)
def rootSettings = sharedSettings ++ Seq(
publish := {}
)
- def replSettings = sharedSettings ++ Seq(
+ def replSettings = sharedSettings ++ Seq(
name := "spark-repl",
- libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _)
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ),
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v ),
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v )
)
def examplesSettings = sharedSettings ++ Seq(
name := "spark-examples",
libraryDependencies ++= Seq(
- "com.twitter" % "algebird-core_2.9.2" % "0.1.11",
-
+ "com.twitter" %% "algebird-core" % "0.1.11",
+ "org.apache.hbase" % "hbase" % "0.94.6" excludeAll(excludeNetty, excludeAsm),
"org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm),
-
- "org.apache.cassandra" % "cassandra-all" % "1.2.5"
+ "org.apache.cassandra" % "cassandra-all" % "1.2.6"
exclude("com.google.guava", "guava")
exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru")
exclude("com.ning","compress-lzf")
@@ -288,26 +325,34 @@ object SparkBuild extends Build {
def streamingSettings = sharedSettings ++ Seq(
name := "spark-streaming",
- resolvers ++= Seq(
- "Akka Repository" at "http://repo.akka.io/releases/",
- "Apache repo" at "https://repository.apache.org/content/repositories/releases"
- ),
libraryDependencies ++= Seq(
- "org.eclipse.paho" % "mqtt-client" % "0.4.0",
- "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy),
- "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty),
- "com.typesafe.akka" % "akka-zeromq" % "2.0.5" excludeAll(excludeNetty),
- "org.apache.kafka" % "kafka_2.9.2" % "0.8.0-beta1"
- exclude("com.sun.jdmk", "jmxtools")
- exclude("com.sun.jmx", "jmxri")
- exclude("net.sf.jopt-simple", "jopt-simple")
+ "commons-io" % "commons-io" % "2.4"
)
)
- def yarnSettings = sharedSettings ++ Seq(
- name := "spark-yarn"
+ def yarnCommonSettings = sharedSettings ++ Seq(
+ unmanagedSourceDirectories in Compile <++= baseDirectory { base =>
+ Seq(
+ base / "../common/src/main/scala"
+ )
+ },
+
+ unmanagedSourceDirectories in Test <++= baseDirectory { base =>
+ Seq(
+ base / "../common/src/test/scala"
+ )
+ }
+
) ++ extraYarnSettings
+ def yarnAlphaSettings = yarnCommonSettings ++ Seq(
+ name := "spark-yarn-alpha"
+ )
+
+ def yarnSettings = yarnCommonSettings ++ Seq(
+ name := "spark-yarn"
+ )
+
// Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain
// if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN).
def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq()
@@ -315,14 +360,15 @@ object SparkBuild extends Build {
def yarnEnabledSettings = Seq(
libraryDependencies ++= Seq(
// Exclude rule required for all ?
- "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-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)
)
)
def assemblyProjSettings = sharedSettings ++ Seq(
+ libraryDependencies += "net.sf.py4j" % "py4j" % "0.8.1",
name := "spark-assembly",
assembleDeps in Compile <<= (packageProjects.map(packageBin in Compile in _) ++ Seq(packageDependency in Compile)).dependOn,
jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" },
@@ -340,4 +386,43 @@ object SparkBuild extends Build {
case _ => MergeStrategy.first
}
)
+
+ def twitterSettings() = sharedSettings ++ Seq(
+ name := "spark-streaming-twitter",
+ libraryDependencies ++= Seq(
+ "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty)
+ )
+ )
+
+ def kafkaSettings() = sharedSettings ++ Seq(
+ name := "spark-streaming-kafka",
+ libraryDependencies ++= Seq(
+ "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty),
+ "com.sksamuel.kafka" %% "kafka" % "0.8.0-beta1"
+ exclude("com.sun.jdmk", "jmxtools")
+ exclude("com.sun.jmx", "jmxri")
+ exclude("net.sf.jopt-simple", "jopt-simple")
+ excludeAll(excludeNetty)
+ )
+ )
+
+ def flumeSettings() = sharedSettings ++ Seq(
+ name := "spark-streaming-flume",
+ libraryDependencies ++= Seq(
+ "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy)
+ )
+ )
+
+ def zeromqSettings() = sharedSettings ++ Seq(
+ name := "spark-streaming-zeromq",
+ libraryDependencies ++= Seq(
+ "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty)
+ )
+ )
+
+ def mqttSettings() = streamingSettings ++ Seq(
+ name := "spark-streaming-mqtt",
+ resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"),
+ libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0")
+ )
}
diff --git a/project/build.properties b/project/build.properties
index 9647277162..839f5fbb0c 100644
--- a/project/build.properties
+++ b/project/build.properties
@@ -14,5 +14,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
-
sbt.version=0.12.4
diff --git a/python/epydoc.conf b/python/epydoc.conf
index 1d0d002d36..95a6af0974 100644
--- a/python/epydoc.conf
+++ b/python/epydoc.conf
@@ -32,6 +32,6 @@ target: docs/
private: no
-exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.serializers
+exclude: pyspark.cloudpickle pyspark.worker pyspark.join
pyspark.java_gateway pyspark.examples pyspark.shell pyspark.test
- pyspark.rddsampler pyspark.daemon
+ pyspark.rddsampler pyspark.daemon pyspark.mllib._common
diff --git a/python/lib/py4j-0.8.1-src.zip b/python/lib/py4j-0.8.1-src.zip
new file mode 100644
index 0000000000..2069a328d1
--- /dev/null
+++ b/python/lib/py4j-0.8.1-src.zip
Binary files differ
diff --git a/python/lib/py4j0.7.egg b/python/lib/py4j0.7.egg
deleted file mode 100644
index f8a339d8ee..0000000000
--- a/python/lib/py4j0.7.egg
+++ /dev/null
Binary files differ
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index 1f35f6f939..a51d5af79b 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -20,28 +20,34 @@ PySpark is the Python API for Spark.
Public classes:
- - L{SparkContext<pyspark.context.SparkContext>}
- Main entry point for Spark functionality.
- - L{RDD<pyspark.rdd.RDD>}
- A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
- - L{Broadcast<pyspark.broadcast.Broadcast>}
- A broadcast variable that gets reused across tasks.
- - L{Accumulator<pyspark.accumulators.Accumulator>}
- 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.
+ - L{SparkContext<pyspark.context.SparkContext>}
+ Main entry point for Spark functionality.
+ - L{RDD<pyspark.rdd.RDD>}
+ A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
+ - L{Broadcast<pyspark.broadcast.Broadcast>}
+ A broadcast variable that gets reused across tasks.
+ - L{Accumulator<pyspark.accumulators.Accumulator>}
+ An "add-only" shared variable that tasks can only add values to.
+ - L{SparkConf<pyspark.conf.SparkConf>}
+ For configuring Spark.
+ - L{SparkFiles<pyspark.files.SparkFiles>}
+ Access files shipped with jobs.
+ - L{StorageLevel<pyspark.storagelevel.StorageLevel>}
+ Finer-grained cache persistence levels.
"""
+
+
+
import sys
import os
-sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.egg"))
+sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j-0.8.1-src.zip"))
+from pyspark.conf import SparkConf
from pyspark.context import SparkContext
from pyspark.rdd import RDD
from pyspark.files import SparkFiles
from pyspark.storagelevel import StorageLevel
-__all__ = ["SparkContext", "RDD", "SparkFiles", "StorageLevel"]
+__all__ = ["SparkConf", "SparkContext", "RDD", "SparkFiles", "StorageLevel"]
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index da3d96689a..2204e9c9ca 100644
--- a/python/pyspark/accumulators.py
+++ b/python/pyspark/accumulators.py
@@ -90,9 +90,11 @@ import struct
import SocketServer
import threading
from pyspark.cloudpickle import CloudPickler
-from pyspark.serializers import read_int, read_with_length, load_pickle
+from pyspark.serializers import read_int, PickleSerializer
+pickleSer = PickleSerializer()
+
# Holds accumulators registered on the current machine, keyed by ID. This is then used to send
# the local accumulator updates back to the driver program at the end of a task.
_accumulatorRegistry = {}
@@ -211,7 +213,7 @@ class _UpdateRequestHandler(SocketServer.StreamRequestHandler):
from pyspark.accumulators import _accumulatorRegistry
num_updates = read_int(self.rfile)
for _ in range(num_updates):
- (aid, update) = load_pickle(read_with_length(self.rfile))
+ (aid, update) = pickleSer._read_with_length(self.rfile)
_accumulatorRegistry[aid] += update
# Write a byte in acknowledgement
self.wfile.write(struct.pack("!b", 1))
diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py
index dfdaba274f..43f40f8783 100644
--- a/python/pyspark/broadcast.py
+++ b/python/pyspark/broadcast.py
@@ -45,7 +45,18 @@ def _from_id(bid):
class Broadcast(object):
+ """
+ A broadcast variable created with
+ L{SparkContext.broadcast()<pyspark.context.SparkContext.broadcast>}.
+ Access its value through C{.value}.
+ """
+
def __init__(self, bid, value, java_broadcast=None, pickle_registry=None):
+ """
+ Should not be called directly by users -- use
+ L{SparkContext.broadcast()<pyspark.context.SparkContext.broadcast>}
+ instead.
+ """
self.value = value
self.bid = bid
self._jbroadcast = java_broadcast
diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py
new file mode 100644
index 0000000000..d72aed6a30
--- /dev/null
+++ b/python/pyspark/conf.py
@@ -0,0 +1,171 @@
+#
+# 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 pyspark.conf import SparkConf
+>>> from pyspark.context import SparkContext
+>>> conf = SparkConf()
+>>> conf.setMaster("local").setAppName("My app")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.master")
+u'local'
+>>> conf.get("spark.app.name")
+u'My app'
+>>> sc = SparkContext(conf=conf)
+>>> sc.master
+u'local'
+>>> sc.appName
+u'My app'
+>>> sc.sparkHome == None
+True
+
+>>> conf = SparkConf()
+>>> conf.setSparkHome("/path")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.home")
+u'/path'
+>>> conf.setExecutorEnv("VAR1", "value1")
+<pyspark.conf.SparkConf object at ...>
+>>> conf.setExecutorEnv(pairs = [("VAR3", "value3"), ("VAR4", "value4")])
+<pyspark.conf.SparkConf object at ...>
+>>> conf.get("spark.executorEnv.VAR1")
+u'value1'
+>>> print conf.toDebugString()
+spark.executorEnv.VAR1=value1
+spark.executorEnv.VAR3=value3
+spark.executorEnv.VAR4=value4
+spark.home=/path
+>>> sorted(conf.getAll(), key=lambda p: p[0])
+[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')]
+"""
+
+
+class SparkConf(object):
+ """
+ Configuration for a Spark application. Used to set various Spark
+ parameters as key-value pairs.
+
+ Most of the time, you would create a SparkConf object with
+ C{SparkConf()}, which will load values from C{spark.*} Java system
+ properties and any C{spark.conf} on your Spark classpath. In this
+ case, system properties take priority over C{spark.conf}, and any
+ parameters you set directly on the C{SparkConf} object take priority
+ over both of those.
+
+ For unit tests, you can also call C{SparkConf(false)} to skip
+ loading external settings and get the same configuration no matter
+ what is on the classpath.
+
+ All setter methods in this class support chaining. For example,
+ you can write C{conf.setMaster("local").setAppName("My app")}.
+
+ Note that once a SparkConf object is passed to Spark, it is cloned
+ and can no longer be modified by the user.
+ """
+
+ def __init__(self, loadDefaults=True, _jvm=None):
+ """
+ Create a new Spark configuration.
+
+ @param loadDefaults: whether to load values from Java system
+ properties and classpath (True by default)
+ @param _jvm: internal parameter used to pass a handle to the
+ Java VM; does not need to be set by users
+ """
+ from pyspark.context import SparkContext
+ SparkContext._ensure_initialized()
+ _jvm = _jvm or SparkContext._jvm
+ self._jconf = _jvm.SparkConf(loadDefaults)
+
+ def set(self, key, value):
+ """Set a configuration property."""
+ self._jconf.set(key, unicode(value))
+ return self
+
+ def setMaster(self, value):
+ """Set master URL to connect to."""
+ self._jconf.setMaster(value)
+ return self
+
+ def setAppName(self, value):
+ """Set application name."""
+ self._jconf.setAppName(value)
+ return self
+
+ def setSparkHome(self, value):
+ """Set path where Spark is installed on worker nodes."""
+ self._jconf.setSparkHome(value)
+ return self
+
+ def setExecutorEnv(self, key=None, value=None, pairs=None):
+ """Set an environment variable to be passed to executors."""
+ if (key != None and pairs != None) or (key == None and pairs == None):
+ raise Exception("Either pass one key-value pair or a list of pairs")
+ elif key != None:
+ self._jconf.setExecutorEnv(key, value)
+ elif pairs != None:
+ for (k, v) in pairs:
+ self._jconf.setExecutorEnv(k, v)
+ return self
+
+ def setAll(self, pairs):
+ """
+ Set multiple parameters, passed as a list of key-value pairs.
+
+ @param pairs: list of key-value pairs to set
+ """
+ for (k, v) in pairs:
+ self._jconf.set(k, v)
+ return self
+
+ def get(self, key, defaultValue=None):
+ """Get the configured value for some key, or return a default otherwise."""
+ if defaultValue == None: # Py4J doesn't call the right get() if we pass None
+ if not self._jconf.contains(key):
+ return None
+ return self._jconf.get(key)
+ else:
+ return self._jconf.get(key, defaultValue)
+
+ def getAll(self):
+ """Get all values as a list of key-value pairs."""
+ pairs = []
+ for elem in self._jconf.getAll():
+ pairs.append((elem._1(), elem._2()))
+ return pairs
+
+ def contains(self, key):
+ """Does this configuration contain a given key?"""
+ return self._jconf.contains(key)
+
+ def toDebugString(self):
+ """
+ Returns a printable version of the configuration, as a list of
+ key=value pairs, one per line.
+ """
+ return self._jconf.toDebugString()
+
+
+def _test():
+ import doctest
+ (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS)
+ if failure_count:
+ exit(-1)
+
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index a7ca8bc888..f955aad7a4 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -24,9 +24,10 @@ from tempfile import NamedTemporaryFile
from pyspark import accumulators
from pyspark.accumulators import Accumulator
from pyspark.broadcast import Broadcast
+from pyspark.conf import SparkConf
from pyspark.files import SparkFiles
from pyspark.java_gateway import launch_gateway
-from pyspark.serializers import dump_pickle, write_with_length, batched
+from pyspark.serializers import PickleSerializer, BatchedSerializer, MUTF8Deserializer
from pyspark.storagelevel import StorageLevel
from pyspark.rdd import RDD
@@ -42,22 +43,22 @@ class SparkContext(object):
_gateway = None
_jvm = None
- _writeIteratorToPickleFile = None
- _takePartition = None
+ _writeToFile = None
_next_accum_id = 0
_active_spark_context = None
_lock = Lock()
_python_includes = None # zip and egg files that need to be added to PYTHONPATH
- def __init__(self, master, jobName, sparkHome=None, pyFiles=None,
- environment=None, batchSize=1024):
+ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
+ environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None):
"""
- Create a new SparkContext.
+ Create a new SparkContext. At least the master and app name should be set,
+ either through the named parameters here or through C{conf}.
@param master: Cluster URL to connect to
(e.g. mesos://host:port, spark://host:port, local[4]).
- @param jobName: A name for your job, to display on the cluster web UI
+ @param appName: A name for your job, to display on the cluster web UI.
@param sparkHome: Location where Spark is installed on cluster nodes.
@param pyFiles: Collection of .zip or .py files to send to the cluster
and add to PYTHONPATH. These can be paths on the local file
@@ -67,6 +68,8 @@ class SparkContext(object):
@param batchSize: The number of Python objects represented as a single
Java object. Set 1 to disable batching or -1 to use an
unlimited batch size.
+ @param serializer: The serializer for RDDs.
+ @param conf: A L{SparkConf} object setting Spark properties.
>>> from pyspark.context import SparkContext
@@ -79,16 +82,45 @@ class SparkContext(object):
"""
SparkContext._ensure_initialized(self)
- self.master = master
- self.jobName = jobName
- self.sparkHome = sparkHome or None # None becomes null in Py4J
self.environment = environment or {}
- self.batchSize = batchSize # -1 represents a unlimited batch size
+ self._conf = conf or SparkConf(_jvm=self._jvm)
+ self._batchSize = batchSize # -1 represents an unlimited batch size
+ self._unbatched_serializer = serializer
+ if batchSize == 1:
+ self.serializer = self._unbatched_serializer
+ else:
+ self.serializer = BatchedSerializer(self._unbatched_serializer,
+ batchSize)
+
+ # Set any parameters passed directly to us on the conf
+ if master:
+ self._conf.setMaster(master)
+ if appName:
+ self._conf.setAppName(appName)
+ if sparkHome:
+ self._conf.setSparkHome(sparkHome)
+ if environment:
+ for key, value in environment.iteritems():
+ self._conf.setExecutorEnv(key, value)
+
+ # Check that we have at least the required parameters
+ if not self._conf.contains("spark.master"):
+ raise Exception("A master URL must be set in your configuration")
+ if not self._conf.contains("spark.app.name"):
+ raise Exception("An application name must be set in your configuration")
+
+ # Read back our properties from the conf in case we loaded some of them from
+ # the classpath or an external config file
+ self.master = self._conf.get("spark.master")
+ self.appName = self._conf.get("spark.app.name")
+ self.sparkHome = self._conf.get("spark.home", None)
+ for (k, v) in self._conf.getAll():
+ if k.startswith("spark.executorEnv."):
+ varName = k[len("spark.executorEnv."):]
+ self.environment[varName] = v
# Create the Java SparkContext through Py4J
- empty_string_array = self._gateway.new_array(self._jvm.String, 0)
- self._jsc = self._jvm.JavaSparkContext(master, jobName, sparkHome,
- empty_string_array)
+ self._jsc = self._jvm.JavaSparkContext(self._conf._jconf)
# Create a single Accumulator in Java that we'll send all our updates through;
# they will be passed back to us through a TCP server
@@ -99,6 +131,7 @@ class SparkContext(object):
self._jvm.PythonAccumulatorParam(host, port))
self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python')
+
# Broadcast's __reduce__ method stores Broadcast instances here.
# This allows other code to determine which Broadcast instances have
# been pickled, so it can determine which Java broadcast objects to
@@ -115,7 +148,7 @@ class SparkContext(object):
self.addPyFile(path)
# Create a temporary directory inside spark.local.dir:
- local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir()
+ local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf())
self._temp_dir = \
self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath()
@@ -125,10 +158,7 @@ class SparkContext(object):
if not SparkContext._gateway:
SparkContext._gateway = launch_gateway()
SparkContext._jvm = SparkContext._gateway.jvm
- SparkContext._writeIteratorToPickleFile = \
- SparkContext._jvm.PythonRDD.writeIteratorToPickleFile
- SparkContext._takePartition = \
- SparkContext._jvm.PythonRDD.takePartition
+ SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile
if instance:
if SparkContext._active_spark_context and SparkContext._active_spark_context != instance:
@@ -139,8 +169,8 @@ class SparkContext(object):
@classmethod
def setSystemProperty(cls, key, value):
"""
- Set a system property, such as spark.executor.memory. This must be
- invoked before instantiating SparkContext.
+ Set a Java system property, such as spark.executor.memory. This must
+ must be invoked before instantiating SparkContext.
"""
SparkContext._ensure_initialized()
SparkContext._jvm.java.lang.System.setProperty(key, value)
@@ -184,15 +214,17 @@ class SparkContext(object):
# Make sure we distribute data evenly if it's smaller than self.batchSize
if "__len__" not in dir(c):
c = list(c) # Make it a list so we can compute its length
- batchSize = min(len(c) // numSlices, self.batchSize)
+ batchSize = min(len(c) // numSlices, self._batchSize)
if batchSize > 1:
- c = batched(c, batchSize)
- for x in c:
- write_with_length(dump_pickle(x), tempFile)
+ serializer = BatchedSerializer(self._unbatched_serializer,
+ batchSize)
+ else:
+ serializer = self._unbatched_serializer
+ serializer.dump_stream(c, tempFile)
tempFile.close()
- readRDDFromPickleFile = self._jvm.PythonRDD.readRDDFromPickleFile
- jrdd = readRDDFromPickleFile(self._jsc, tempFile.name, numSlices)
- return RDD(jrdd, self)
+ readRDDFromFile = self._jvm.PythonRDD.readRDDFromFile
+ jrdd = readRDDFromFile(self._jsc, tempFile.name, numSlices)
+ return RDD(jrdd, self, serializer)
def textFile(self, name, minSplits=None):
"""
@@ -201,29 +233,50 @@ class SparkContext(object):
RDD of Strings.
"""
minSplits = minSplits or min(self.defaultParallelism, 2)
- jrdd = self._jsc.textFile(name, minSplits)
- return RDD(jrdd, self)
+ return RDD(self._jsc.textFile(name, minSplits), self,
+ MUTF8Deserializer())
- def _checkpointFile(self, name):
+ def _checkpointFile(self, name, input_deserializer):
jrdd = self._jsc.checkpointFile(name)
- return RDD(jrdd, self)
+ return RDD(jrdd, self, input_deserializer)
def union(self, rdds):
"""
Build the union of a list of RDDs.
+
+ This supports unions() of RDDs with different serialized formats,
+ although this forces them to be reserialized using the default
+ serializer:
+
+ >>> path = os.path.join(tempdir, "union-text.txt")
+ >>> with open(path, "w") as testFile:
+ ... testFile.write("Hello")
+ >>> textFile = sc.textFile(path)
+ >>> textFile.collect()
+ [u'Hello']
+ >>> parallelized = sc.parallelize(["World!"])
+ >>> sorted(sc.union([textFile, parallelized]).collect())
+ [u'Hello', 'World!']
"""
+ first_jrdd_deserializer = rdds[0]._jrdd_deserializer
+ if any(x._jrdd_deserializer != first_jrdd_deserializer for x in rdds):
+ rdds = [x._reserialize() for x in rdds]
first = rdds[0]._jrdd
rest = [x._jrdd for x in rdds[1:]]
- rest = ListConverter().convert(rest, self.gateway._gateway_client)
- return RDD(self._jsc.union(first, rest), self)
+ rest = ListConverter().convert(rest, self._gateway._gateway_client)
+ return RDD(self._jsc.union(first, rest), self,
+ rdds[0]._jrdd_deserializer)
def broadcast(self, value):
"""
- Broadcast a read-only variable to the cluster, returning a C{Broadcast}
+ Broadcast a read-only variable to the cluster, returning a
+ L{Broadcast<pyspark.broadcast.Broadcast>}
object for reading it in distributed functions. The variable will be
sent to each cluster only once.
"""
- jbroadcast = self._jsc.broadcast(bytearray(dump_pickle(value)))
+ pickleSer = PickleSerializer()
+ pickled = pickleSer.dumps(value)
+ jbroadcast = self._jsc.broadcast(bytearray(pickled))
return Broadcast(jbroadcast.id(), value, jbroadcast,
self._pickled_broadcast_vars)
@@ -235,7 +288,7 @@ class SparkContext(object):
and floating-point numbers if you do not provide one. For other types,
a custom AccumulatorParam can be used.
"""
- if accum_param == None:
+ if accum_param is None:
if isinstance(value, int):
accum_param = accumulators.INT_ACCUMULATOR_PARAM
elif isinstance(value, float):
@@ -294,17 +347,12 @@ class SparkContext(object):
self._python_includes.append(filename)
sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) # for tests in local mode
- def setCheckpointDir(self, dirName, useExisting=False):
+ def setCheckpointDir(self, dirName):
"""
Set the directory under which RDDs are going to be checkpointed. The
directory must be a HDFS path if running on a cluster.
-
- If the directory does not exist, it will be created. If the directory
- exists and C{useExisting} is set to true, then the exisiting directory
- will be used. Otherwise an exception will be thrown to prevent
- accidental overriding of checkpoint files in the existing directory.
"""
- self._jsc.sc().setCheckpointDir(dirName, useExisting)
+ self._jsc.sc().setCheckpointDir(dirName)
def _getJavaStorageLevel(self, storageLevel):
"""
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index e615c1e9b6..c15add5237 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -31,7 +31,7 @@ def launch_gateway():
# Launch the Py4j gateway using Spark's run command so that we pick up the
# proper classpath and SPARK_MEM settings from spark-env.sh
on_windows = platform.system() == "Windows"
- script = "spark-class.cmd" if on_windows else "spark-class"
+ script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
"--die-on-broken-pipe", "0"]
if not on_windows:
@@ -60,7 +60,9 @@ def launch_gateway():
# Connect to the gateway
gateway = JavaGateway(GatewayClient(port=port), auto_convert=False)
# Import the classes used by PySpark
+ java_import(gateway.jvm, "org.apache.spark.SparkConf")
java_import(gateway.jvm, "org.apache.spark.api.java.*")
java_import(gateway.jvm, "org.apache.spark.api.python.*")
+ java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*")
java_import(gateway.jvm, "scala.Tuple2")
return gateway
diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py
new file mode 100644
index 0000000000..b1a5df109b
--- /dev/null
+++ b/python/pyspark/mllib/__init__.py
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+Python bindings for MLlib.
+"""
diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py
new file mode 100644
index 0000000000..769d88dfb9
--- /dev/null
+++ b/python/pyspark/mllib/_common.py
@@ -0,0 +1,252 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape
+from pyspark import SparkContext
+
+from pyspark.serializers import Serializer
+import struct
+
+# Double vector format:
+#
+# [8-byte 1] [8-byte length] [length*8 bytes of data]
+#
+# Double matrix format:
+#
+# [8-byte 2] [8-byte rows] [8-byte cols] [rows*cols*8 bytes of data]
+#
+# This is all in machine-endian. That means that the Java interpreter and the
+# Python interpreter must agree on what endian the machine is.
+
+def _deserialize_byte_array(shape, ba, offset):
+ """Wrapper around ndarray aliasing hack.
+
+ >>> x = array([1.0, 2.0, 3.0, 4.0, 5.0])
+ >>> array_equal(x, _deserialize_byte_array(x.shape, x.data, 0))
+ True
+ >>> x = array([1.0, 2.0, 3.0, 4.0]).reshape(2,2)
+ >>> array_equal(x, _deserialize_byte_array(x.shape, x.data, 0))
+ True
+ """
+ ar = ndarray(shape=shape, buffer=ba, offset=offset, dtype="float64",
+ order='C')
+ return ar.copy()
+
+def _serialize_double_vector(v):
+ """Serialize a double vector into a mutually understood format."""
+ if type(v) != ndarray:
+ raise TypeError("_serialize_double_vector called on a %s; "
+ "wanted ndarray" % type(v))
+ if v.dtype != float64:
+ raise TypeError("_serialize_double_vector called on an ndarray of %s; "
+ "wanted ndarray of float64" % v.dtype)
+ if v.ndim != 1:
+ raise TypeError("_serialize_double_vector called on a %ddarray; "
+ "wanted a 1darray" % v.ndim)
+ length = v.shape[0]
+ ba = bytearray(16 + 8*length)
+ header = ndarray(shape=[2], buffer=ba, dtype="int64")
+ header[0] = 1
+ header[1] = length
+ copyto(ndarray(shape=[length], buffer=ba, offset=16,
+ dtype="float64"), v)
+ return ba
+
+def _deserialize_double_vector(ba):
+ """Deserialize a double vector from a mutually understood format.
+
+ >>> x = array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0])
+ >>> array_equal(x, _deserialize_double_vector(_serialize_double_vector(x)))
+ True
+ """
+ if type(ba) != bytearray:
+ raise TypeError("_deserialize_double_vector called on a %s; "
+ "wanted bytearray" % type(ba))
+ if len(ba) < 16:
+ raise TypeError("_deserialize_double_vector called on a %d-byte array, "
+ "which is too short" % len(ba))
+ if (len(ba) & 7) != 0:
+ raise TypeError("_deserialize_double_vector called on a %d-byte array, "
+ "which is not a multiple of 8" % len(ba))
+ header = ndarray(shape=[2], buffer=ba, dtype="int64")
+ if header[0] != 1:
+ raise TypeError("_deserialize_double_vector called on bytearray "
+ "with wrong magic")
+ length = header[1]
+ if len(ba) != 8*length + 16:
+ raise TypeError("_deserialize_double_vector called on bytearray "
+ "with wrong length")
+ return _deserialize_byte_array([length], ba, 16)
+
+def _serialize_double_matrix(m):
+ """Serialize a double matrix into a mutually understood format."""
+ if (type(m) == ndarray and m.dtype == float64 and m.ndim == 2):
+ rows = m.shape[0]
+ cols = m.shape[1]
+ ba = bytearray(24 + 8 * rows * cols)
+ header = ndarray(shape=[3], buffer=ba, dtype="int64")
+ header[0] = 2
+ header[1] = rows
+ header[2] = cols
+ copyto(ndarray(shape=[rows, cols], buffer=ba, offset=24,
+ dtype="float64", order='C'), m)
+ return ba
+ else:
+ raise TypeError("_serialize_double_matrix called on a "
+ "non-double-matrix")
+
+def _deserialize_double_matrix(ba):
+ """Deserialize a double matrix from a mutually understood format."""
+ if type(ba) != bytearray:
+ raise TypeError("_deserialize_double_matrix called on a %s; "
+ "wanted bytearray" % type(ba))
+ if len(ba) < 24:
+ raise TypeError("_deserialize_double_matrix called on a %d-byte array, "
+ "which is too short" % len(ba))
+ if (len(ba) & 7) != 0:
+ raise TypeError("_deserialize_double_matrix called on a %d-byte array, "
+ "which is not a multiple of 8" % len(ba))
+ header = ndarray(shape=[3], buffer=ba, dtype="int64")
+ if (header[0] != 2):
+ raise TypeError("_deserialize_double_matrix called on bytearray "
+ "with wrong magic")
+ rows = header[1]
+ cols = header[2]
+ if (len(ba) != 8*rows*cols + 24):
+ raise TypeError("_deserialize_double_matrix called on bytearray "
+ "with wrong length")
+ return _deserialize_byte_array([rows, cols], ba, 24)
+
+def _linear_predictor_typecheck(x, coeffs):
+ """Check that x is a one-dimensional vector of the right shape.
+ This is a temporary hackaround until I actually implement bulk predict."""
+ if type(x) == ndarray:
+ if x.ndim == 1:
+ if x.shape == coeffs.shape:
+ pass
+ else:
+ raise RuntimeError("Got array of %d elements; wanted %d"
+ % (shape(x)[0], shape(coeffs)[0]))
+ else:
+ raise RuntimeError("Bulk predict not yet supported.")
+ elif (type(x) == RDD):
+ raise RuntimeError("Bulk predict not yet supported.")
+ else:
+ raise TypeError("Argument of type " + type(x).__name__ + " unsupported")
+
+def _get_unmangled_rdd(data, serializer):
+ dataBytes = data.map(serializer)
+ dataBytes._bypass_serializer = True
+ dataBytes.cache()
+ return dataBytes
+
+# Map a pickled Python RDD of numpy double vectors to a Java RDD of
+# _serialized_double_vectors
+def _get_unmangled_double_vector_rdd(data):
+ return _get_unmangled_rdd(data, _serialize_double_vector)
+
+class LinearModel(object):
+ """Something that has a vector of coefficients and an intercept."""
+ def __init__(self, coeff, intercept):
+ self._coeff = coeff
+ self._intercept = intercept
+
+class LinearRegressionModelBase(LinearModel):
+ """A linear regression model.
+
+ >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1)
+ >>> abs(lrmb.predict(array([-1.03, 7.777])) - 14.624) < 1e-6
+ True
+ """
+ def predict(self, x):
+ """Predict the value of the dependent variable given a vector x"""
+ """containing values for the independent variables."""
+ _linear_predictor_typecheck(x, self._coeff)
+ return dot(self._coeff, x) + self._intercept
+
+# If we weren't given initial weights, take a zero vector of the appropriate
+# length.
+def _get_initial_weights(initial_weights, data):
+ if initial_weights is None:
+ initial_weights = data.first()
+ if type(initial_weights) != ndarray:
+ raise TypeError("At least one data element has type "
+ + type(initial_weights).__name__ + " which is not ndarray")
+ if initial_weights.ndim != 1:
+ raise TypeError("At least one data element has "
+ + initial_weights.ndim + " dimensions, which is not 1")
+ initial_weights = ones([initial_weights.shape[0] - 1])
+ return initial_weights
+
+# train_func should take two parameters, namely data and initial_weights, and
+# return the result of a call to the appropriate JVM stub.
+# _regression_train_wrapper is responsible for setup and error checking.
+def _regression_train_wrapper(sc, train_func, klass, data, initial_weights):
+ initial_weights = _get_initial_weights(initial_weights, data)
+ dataBytes = _get_unmangled_double_vector_rdd(data)
+ ans = train_func(dataBytes, _serialize_double_vector(initial_weights))
+ if len(ans) != 2:
+ raise RuntimeError("JVM call result had unexpected length")
+ elif type(ans[0]) != bytearray:
+ raise RuntimeError("JVM call result had first element of type "
+ + type(ans[0]).__name__ + " which is not bytearray")
+ elif type(ans[1]) != float:
+ raise RuntimeError("JVM call result had second element of type "
+ + type(ans[0]).__name__ + " which is not float")
+ return klass(_deserialize_double_vector(ans[0]), ans[1])
+
+def _serialize_rating(r):
+ ba = bytearray(16)
+ intpart = ndarray(shape=[2], buffer=ba, dtype=int32)
+ doublepart = ndarray(shape=[1], buffer=ba, dtype=float64, offset=8)
+ intpart[0], intpart[1], doublepart[0] = r
+ return ba
+
+class RatingDeserializer(Serializer):
+ def loads(self, stream):
+ length = struct.unpack("!i", stream.read(4))[0]
+ ba = stream.read(length)
+ res = ndarray(shape=(3, ), buffer=ba, dtype="float64", offset=4)
+ return int(res[0]), int(res[1]), res[2]
+
+ def load_stream(self, stream):
+ while True:
+ try:
+ yield self.loads(stream)
+ except struct.error:
+ return
+ except EOFError:
+ return
+
+def _serialize_tuple(t):
+ ba = bytearray(8)
+ intpart = ndarray(shape=[2], buffer=ba, dtype=int32)
+ intpart[0], intpart[1] = t
+ return ba
+
+def _test():
+ import doctest
+ globs = globals().copy()
+ globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+ (failure_count, test_count) = doctest.testmod(globs=globs,
+ optionflags=doctest.ELLIPSIS)
+ globs['sc'].stop()
+ if failure_count:
+ exit(-1)
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py
new file mode 100644
index 0000000000..70de332d34
--- /dev/null
+++ b/python/pyspark/mllib/classification.py
@@ -0,0 +1,86 @@
+#
+# 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 numpy import array, dot, shape
+from pyspark import SparkContext
+from pyspark.mllib._common import \
+ _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+ _serialize_double_matrix, _deserialize_double_matrix, \
+ _serialize_double_vector, _deserialize_double_vector, \
+ _get_initial_weights, _serialize_rating, _regression_train_wrapper, \
+ LinearModel, _linear_predictor_typecheck
+from math import exp, log
+
+class LogisticRegressionModel(LinearModel):
+ """A linear binary classification model derived from logistic regression.
+
+ >>> data = array([0.0, 0.0, 1.0, 1.0, 1.0, 2.0, 1.0, 3.0]).reshape(4,2)
+ >>> lrm = LogisticRegressionWithSGD.train(sc, sc.parallelize(data))
+ >>> lrm.predict(array([1.0])) != None
+ True
+ """
+ def predict(self, x):
+ _linear_predictor_typecheck(x, self._coeff)
+ margin = dot(x, self._coeff) + self._intercept
+ prob = 1/(1 + exp(-margin))
+ return 1 if prob > 0.5 else 0
+
+class LogisticRegressionWithSGD(object):
+ @classmethod
+ def train(cls, sc, data, iterations=100, step=1.0,
+ mini_batch_fraction=1.0, initial_weights=None):
+ """Train a logistic regression model on the given data."""
+ return _regression_train_wrapper(sc, lambda d, i:
+ sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD(d._jrdd,
+ iterations, step, mini_batch_fraction, i),
+ LogisticRegressionModel, data, initial_weights)
+
+class SVMModel(LinearModel):
+ """A support vector machine.
+
+ >>> data = array([0.0, 0.0, 1.0, 1.0, 1.0, 2.0, 1.0, 3.0]).reshape(4,2)
+ >>> svm = SVMWithSGD.train(sc, sc.parallelize(data))
+ >>> svm.predict(array([1.0])) != None
+ True
+ """
+ def predict(self, x):
+ _linear_predictor_typecheck(x, self._coeff)
+ margin = dot(x, self._coeff) + self._intercept
+ return 1 if margin >= 0 else 0
+
+class SVMWithSGD(object):
+ @classmethod
+ def train(cls, sc, data, iterations=100, step=1.0, reg_param=1.0,
+ mini_batch_fraction=1.0, initial_weights=None):
+ """Train a support vector machine on the given data."""
+ return _regression_train_wrapper(sc, lambda d, i:
+ sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD(d._jrdd,
+ iterations, step, reg_param, mini_batch_fraction, i),
+ SVMModel, data, initial_weights)
+
+def _test():
+ import doctest
+ globs = globals().copy()
+ globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+ (failure_count, test_count) = doctest.testmod(globs=globs,
+ optionflags=doctest.ELLIPSIS)
+ globs['sc'].stop()
+ if failure_count:
+ exit(-1)
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py
new file mode 100644
index 0000000000..8cf20e591a
--- /dev/null
+++ b/python/pyspark/mllib/clustering.py
@@ -0,0 +1,79 @@
+#
+# 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 numpy import array, dot
+from math import sqrt
+from pyspark import SparkContext
+from pyspark.mllib._common import \
+ _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+ _serialize_double_matrix, _deserialize_double_matrix, \
+ _serialize_double_vector, _deserialize_double_vector, \
+ _get_initial_weights, _serialize_rating, _regression_train_wrapper
+
+class KMeansModel(object):
+ """A clustering model derived from the k-means method.
+
+ >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2)
+ >>> clusters = KMeans.train(sc, sc.parallelize(data), 2, maxIterations=10, runs=30, initialization_mode="random")
+ >>> clusters.predict(array([0.0, 0.0])) == clusters.predict(array([1.0, 1.0]))
+ True
+ >>> clusters.predict(array([8.0, 9.0])) == clusters.predict(array([9.0, 8.0]))
+ True
+ >>> clusters = KMeans.train(sc, sc.parallelize(data), 2)
+ """
+ def __init__(self, centers_):
+ self.centers = centers_
+
+ def predict(self, x):
+ """Find the cluster to which x belongs in this model."""
+ best = 0
+ best_distance = 1e75
+ for i in range(0, self.centers.shape[0]):
+ diff = x - self.centers[i]
+ distance = sqrt(dot(diff, diff))
+ if distance < best_distance:
+ best = i
+ best_distance = distance
+ return best
+
+class KMeans(object):
+ @classmethod
+ def train(cls, sc, data, k, maxIterations=100, runs=1,
+ initialization_mode="k-means||"):
+ """Train a k-means clustering model."""
+ dataBytes = _get_unmangled_double_vector_rdd(data)
+ ans = sc._jvm.PythonMLLibAPI().trainKMeansModel(dataBytes._jrdd,
+ k, maxIterations, runs, initialization_mode)
+ if len(ans) != 1:
+ raise RuntimeError("JVM call result had unexpected length")
+ elif type(ans[0]) != bytearray:
+ raise RuntimeError("JVM call result had first element of type "
+ + type(ans[0]) + " which is not bytearray")
+ return KMeansModel(_deserialize_double_matrix(ans[0]))
+
+def _test():
+ import doctest
+ globs = globals().copy()
+ globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+ (failure_count, test_count) = doctest.testmod(globs=globs,
+ optionflags=doctest.ELLIPSIS)
+ globs['sc'].stop()
+ if failure_count:
+ exit(-1)
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py
new file mode 100644
index 0000000000..0eeb5bb66b
--- /dev/null
+++ b/python/pyspark/mllib/recommendation.py
@@ -0,0 +1,84 @@
+#
+# 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 pyspark import SparkContext
+from pyspark.mllib._common import \
+ _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+ _serialize_double_matrix, _deserialize_double_matrix, \
+ _serialize_double_vector, _deserialize_double_vector, \
+ _get_initial_weights, _serialize_rating, _regression_train_wrapper, \
+ _serialize_tuple, RatingDeserializer
+from pyspark.rdd import RDD
+
+class MatrixFactorizationModel(object):
+ """A matrix factorisation model trained by regularized alternating
+ least-squares.
+
+ >>> r1 = (1, 1, 1.0)
+ >>> r2 = (1, 2, 2.0)
+ >>> r3 = (2, 1, 2.0)
+ >>> ratings = sc.parallelize([r1, r2, r3])
+ >>> model = ALS.trainImplicit(sc, ratings, 1)
+ >>> model.predict(2,2) is not None
+ True
+ >>> testset = sc.parallelize([(1, 2), (1, 1)])
+ >>> model.predictAll(testset).count == 2
+ True
+ """
+
+ def __init__(self, sc, java_model):
+ self._context = sc
+ self._java_model = java_model
+
+ def __del__(self):
+ self._context._gateway.detach(self._java_model)
+
+ def predict(self, user, product):
+ return self._java_model.predict(user, product)
+
+ def predictAll(self, usersProducts):
+ usersProductsJRDD = _get_unmangled_rdd(usersProducts, _serialize_tuple)
+ return RDD(self._java_model.predict(usersProductsJRDD._jrdd),
+ self._context, RatingDeserializer())
+
+class ALS(object):
+ @classmethod
+ def train(cls, sc, ratings, rank, iterations=5, lambda_=0.01, blocks=-1):
+ ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating)
+ mod = sc._jvm.PythonMLLibAPI().trainALSModel(ratingBytes._jrdd,
+ rank, iterations, lambda_, blocks)
+ return MatrixFactorizationModel(sc, mod)
+
+ @classmethod
+ def trainImplicit(cls, sc, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01):
+ ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating)
+ mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel(ratingBytes._jrdd,
+ rank, iterations, lambda_, blocks, alpha)
+ return MatrixFactorizationModel(sc, mod)
+
+def _test():
+ import doctest
+ globs = globals().copy()
+ globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+ (failure_count, test_count) = doctest.testmod(globs=globs,
+ optionflags=doctest.ELLIPSIS)
+ globs['sc'].stop()
+ if failure_count:
+ exit(-1)
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py
new file mode 100644
index 0000000000..a3a68b29e0
--- /dev/null
+++ b/python/pyspark/mllib/regression.py
@@ -0,0 +1,110 @@
+#
+# 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 numpy import array, dot
+from pyspark import SparkContext
+from pyspark.mllib._common import \
+ _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \
+ _serialize_double_matrix, _deserialize_double_matrix, \
+ _serialize_double_vector, _deserialize_double_vector, \
+ _get_initial_weights, _serialize_rating, _regression_train_wrapper, \
+ _linear_predictor_typecheck
+
+class LinearModel(object):
+ """Something that has a vector of coefficients and an intercept."""
+ def __init__(self, coeff, intercept):
+ self._coeff = coeff
+ self._intercept = intercept
+
+class LinearRegressionModelBase(LinearModel):
+ """A linear regression model.
+
+ >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1)
+ >>> abs(lrmb.predict(array([-1.03, 7.777])) - 14.624) < 1e-6
+ True
+ """
+ def predict(self, x):
+ """Predict the value of the dependent variable given a vector x"""
+ """containing values for the independent variables."""
+ _linear_predictor_typecheck(x, self._coeff)
+ return dot(self._coeff, x) + self._intercept
+
+class LinearRegressionModel(LinearRegressionModelBase):
+ """A linear regression model derived from a least-squares fit.
+
+ >>> data = array([0.0, 0.0, 1.0, 1.0, 3.0, 2.0, 2.0, 3.0]).reshape(4,2)
+ >>> lrm = LinearRegressionWithSGD.train(sc, sc.parallelize(data), initial_weights=array([1.0]))
+ """
+
+class LinearRegressionWithSGD(object):
+ @classmethod
+ def train(cls, sc, data, iterations=100, step=1.0,
+ mini_batch_fraction=1.0, initial_weights=None):
+ """Train a linear regression model on the given data."""
+ return _regression_train_wrapper(sc, lambda d, i:
+ sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD(
+ d._jrdd, iterations, step, mini_batch_fraction, i),
+ LinearRegressionModel, data, initial_weights)
+
+class LassoModel(LinearRegressionModelBase):
+ """A linear regression model derived from a least-squares fit with an
+ l_1 penalty term.
+
+ >>> data = array([0.0, 0.0, 1.0, 1.0, 3.0, 2.0, 2.0, 3.0]).reshape(4,2)
+ >>> lrm = LassoWithSGD.train(sc, sc.parallelize(data), initial_weights=array([1.0]))
+ """
+
+class LassoWithSGD(object):
+ @classmethod
+ def train(cls, sc, data, iterations=100, step=1.0, reg_param=1.0,
+ mini_batch_fraction=1.0, initial_weights=None):
+ """Train a Lasso regression model on the given data."""
+ return _regression_train_wrapper(sc, lambda d, i:
+ sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD(d._jrdd,
+ iterations, step, reg_param, mini_batch_fraction, i),
+ LassoModel, data, initial_weights)
+
+class RidgeRegressionModel(LinearRegressionModelBase):
+ """A linear regression model derived from a least-squares fit with an
+ l_2 penalty term.
+
+ >>> data = array([0.0, 0.0, 1.0, 1.0, 3.0, 2.0, 2.0, 3.0]).reshape(4,2)
+ >>> lrm = RidgeRegressionWithSGD.train(sc, sc.parallelize(data), initial_weights=array([1.0]))
+ """
+
+class RidgeRegressionWithSGD(object):
+ @classmethod
+ def train(cls, sc, data, iterations=100, step=1.0, reg_param=1.0,
+ mini_batch_fraction=1.0, initial_weights=None):
+ """Train a ridge regression model on the given data."""
+ return _regression_train_wrapper(sc, lambda d, i:
+ sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD(d._jrdd,
+ iterations, step, reg_param, mini_batch_fraction, i),
+ RidgeRegressionModel, data, initial_weights)
+
+def _test():
+ import doctest
+ globs = globals().copy()
+ globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2)
+ (failure_count, test_count) = doctest.testmod(globs=globs,
+ optionflags=doctest.ELLIPSIS)
+ globs['sc'].stop()
+ if failure_count:
+ exit(-1)
+
+if __name__ == "__main__":
+ _test()
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 7019fb8bee..6fb4a7b3be 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -18,18 +18,18 @@
from base64 import standard_b64encode as b64enc
import copy
from collections import defaultdict
-from itertools import chain, ifilter, imap, product
+from itertools import chain, ifilter, imap
import operator
import os
import sys
import shlex
+import traceback
from subprocess import Popen, PIPE
from tempfile import NamedTemporaryFile
from threading import Thread
-from pyspark import cloudpickle
-from pyspark.serializers import batched, Batch, dump_pickle, load_pickle, \
- read_from_pickle_file, pack_long
+from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
+ BatchedSerializer, CloudPickleSerializer, pack_long
from pyspark.join import python_join, python_left_outer_join, \
python_right_outer_join, python_cogroup
from pyspark.statcounter import StatCounter
@@ -40,6 +40,46 @@ from py4j.java_collections import ListConverter, MapConverter
__all__ = ["RDD"]
+def _extract_concise_traceback():
+ tb = traceback.extract_stack()
+ if len(tb) == 0:
+ return "I'm lost!"
+ # HACK: This function is in a file called 'rdd.py' in the top level of
+ # everything PySpark. Just trim off the directory name and assume
+ # everything in that tree is PySpark guts.
+ file, line, module, what = tb[len(tb) - 1]
+ sparkpath = os.path.dirname(file)
+ first_spark_frame = len(tb) - 1
+ for i in range(0, len(tb)):
+ file, line, fun, what = tb[i]
+ if file.startswith(sparkpath):
+ first_spark_frame = i
+ break
+ if first_spark_frame == 0:
+ file, line, fun, what = tb[0]
+ return "%s at %s:%d" % (fun, file, line)
+ sfile, sline, sfun, swhat = tb[first_spark_frame]
+ ufile, uline, ufun, uwhat = tb[first_spark_frame-1]
+ return "%s at %s:%d" % (sfun, ufile, uline)
+
+_spark_stack_depth = 0
+
+class _JavaStackTrace(object):
+ def __init__(self, sc):
+ self._traceback = _extract_concise_traceback()
+ self._context = sc
+
+ def __enter__(self):
+ global _spark_stack_depth
+ if _spark_stack_depth == 0:
+ self._context._jsc.setCallSite(self._traceback)
+ _spark_stack_depth += 1
+
+ def __exit__(self, type, value, tb):
+ global _spark_stack_depth
+ _spark_stack_depth -= 1
+ if _spark_stack_depth == 0:
+ self._context._jsc.setCallSite(None)
class RDD(object):
"""
@@ -48,12 +88,15 @@ class RDD(object):
operated on in parallel.
"""
- def __init__(self, jrdd, ctx):
+ def __init__(self, jrdd, ctx, jrdd_deserializer):
self._jrdd = jrdd
self.is_cached = False
self.is_checkpointed = False
self.ctx = ctx
- self._partitionFunc = None
+ self._jrdd_deserializer = jrdd_deserializer
+
+ def __repr__(self):
+ return self._jrdd.toString()
@property
def context(self):
@@ -247,7 +290,23 @@ class RDD(object):
>>> rdd.union(rdd).collect()
[1, 1, 2, 3, 1, 1, 2, 3]
"""
- return RDD(self._jrdd.union(other._jrdd), self.ctx)
+ if self._jrdd_deserializer == other._jrdd_deserializer:
+ rdd = RDD(self._jrdd.union(other._jrdd), self.ctx,
+ self._jrdd_deserializer)
+ return rdd
+ else:
+ # These RDDs contain data in different serialized formats, so we
+ # must normalize them to the default serializer.
+ self_copy = self._reserialize()
+ other_copy = other._reserialize()
+ return RDD(self_copy._jrdd.union(other_copy._jrdd), self.ctx,
+ self.ctx.serializer)
+
+ def _reserialize(self):
+ if self._jrdd_deserializer == self.ctx.serializer:
+ return self
+ else:
+ return self.map(lambda x: x, preservesPartitioning=True)
def __add__(self, other):
"""
@@ -334,17 +393,9 @@ class RDD(object):
[(1, 1), (1, 2), (2, 1), (2, 2)]
"""
# Due to batching, we can't use the Java cartesian method.
- java_cartesian = RDD(self._jrdd.cartesian(other._jrdd), self.ctx)
- def unpack_batches(pair):
- (x, y) = pair
- if type(x) == Batch or type(y) == Batch:
- xs = x.items if type(x) == Batch else [x]
- ys = y.items if type(y) == Batch else [y]
- for pair in product(xs, ys):
- yield pair
- else:
- yield pair
- return java_cartesian.flatMap(unpack_batches)
+ deserializer = CartesianDeserializer(self._jrdd_deserializer,
+ other._jrdd_deserializer)
+ return RDD(self._jrdd.cartesian(other._jrdd), self.ctx, deserializer)
def groupBy(self, f, numPartitions=None):
"""
@@ -391,8 +442,9 @@ class RDD(object):
"""
Return a list that contains all of the elements in this RDD.
"""
- picklesInJava = self._jrdd.collect().iterator()
- return list(self._collect_iterator_through_file(picklesInJava))
+ with _JavaStackTrace(self.context) as st:
+ bytesInJava = self._jrdd.collect().iterator()
+ return list(self._collect_iterator_through_file(bytesInJava))
def _collect_iterator_through_file(self, iterator):
# Transferring lots of data through Py4J can be slow because
@@ -400,10 +452,10 @@ class RDD(object):
# file and read it back.
tempFile = NamedTemporaryFile(delete=False, dir=self.ctx._temp_dir)
tempFile.close()
- self.ctx._writeIteratorToPickleFile(iterator, tempFile.name)
+ self.ctx._writeToFile(iterator, tempFile.name)
# Read the data into Python and deserialize it:
with open(tempFile.name, 'rb') as tempFile:
- for item in read_from_pickle_file(tempFile):
+ for item in self._jrdd_deserializer.load_stream(tempFile):
yield item
os.unlink(tempFile.name)
@@ -569,11 +621,17 @@ class RDD(object):
# Take only up to num elements from each partition we try
mapped = self.mapPartitions(takeUpToNum)
items = []
- for partition in range(mapped._jrdd.splits().size()):
- iterator = self.ctx._takePartition(mapped._jrdd.rdd(), partition)
- items.extend(self._collect_iterator_through_file(iterator))
- if len(items) >= num:
- break
+ # TODO(shivaram): Similar to the scala implementation, update the take
+ # method to scan multiple splits based on an estimate of how many elements
+ # we have per-split.
+ with _JavaStackTrace(self.context) as st:
+ for partition in range(mapped._jrdd.splits().size()):
+ partitionsToTake = self.ctx._gateway.new_array(self.ctx._jvm.int, 1)
+ partitionsToTake[0] = partition
+ iterator = mapped._jrdd.collectPartitions(partitionsToTake)[0].iterator()
+ items.extend(mapped._collect_iterator_through_file(iterator))
+ if len(items) >= num:
+ break
return items[:num]
def first(self):
@@ -598,7 +656,10 @@ class RDD(object):
'0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n'
"""
def func(split, iterator):
- return (str(x).encode("utf-8") for x in iterator)
+ for x in iterator:
+ if not isinstance(x, basestring):
+ x = unicode(x)
+ yield x.encode("utf-8")
keyed = PipelinedRDD(self, func)
keyed._bypass_serializer = True
keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path)
@@ -735,6 +796,7 @@ class RDD(object):
# Transferring O(n) objects to Java is too expensive. Instead, we'll
# form the hash buckets in Python, transferring O(numPartitions) objects
# to Java. Each object is a (splitNumber, [objects]) pair.
+ outputSerializer = self.ctx._unbatched_serializer
def add_shuffle_key(split, iterator):
buckets = defaultdict(list)
@@ -743,14 +805,15 @@ class RDD(object):
buckets[partitionFunc(k) % numPartitions].append((k, v))
for (split, items) in buckets.iteritems():
yield pack_long(split)
- yield dump_pickle(Batch(items))
+ yield outputSerializer.dumps(items)
keyed = PipelinedRDD(self, add_shuffle_key)
keyed._bypass_serializer = True
- pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
- partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
- id(partitionFunc))
+ with _JavaStackTrace(self.context) as st:
+ pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD()
+ partitioner = self.ctx._jvm.PythonPartitioner(numPartitions,
+ id(partitionFunc))
jrdd = pairRDD.partitionBy(partitioner).values()
- rdd = RDD(jrdd, self.ctx)
+ rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer))
# This is required so that id(partitionFunc) remains unique, even if
# partitionFunc is a lambda:
rdd._partitionFunc = partitionFunc
@@ -787,7 +850,8 @@ class RDD(object):
numPartitions = self.ctx.defaultParallelism
def combineLocally(iterator):
combiners = {}
- for (k, v) in iterator:
+ for x in iterator:
+ (k, v) = x
if k not in combiners:
combiners[k] = createCombiner(v)
else:
@@ -929,50 +993,52 @@ class PipelinedRDD(RDD):
20
"""
def __init__(self, prev, func, preservesPartitioning=False):
- if isinstance(prev, PipelinedRDD) and prev._is_pipelinable():
+ if not isinstance(prev, PipelinedRDD) or not prev._is_pipelinable():
+ # This transformation is the first in its stage:
+ self.func = func
+ self.preservesPartitioning = preservesPartitioning
+ self._prev_jrdd = prev._jrdd
+ self._prev_jrdd_deserializer = prev._jrdd_deserializer
+ else:
prev_func = prev.func
def pipeline_func(split, iterator):
return func(split, prev_func(split, iterator))
self.func = pipeline_func
self.preservesPartitioning = \
prev.preservesPartitioning and preservesPartitioning
- self._prev_jrdd = prev._prev_jrdd
- else:
- self.func = func
- self.preservesPartitioning = preservesPartitioning
- self._prev_jrdd = prev._jrdd
+ self._prev_jrdd = prev._prev_jrdd # maintain the pipeline
+ self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer
self.is_cached = False
self.is_checkpointed = False
self.ctx = prev.ctx
self.prev = prev
self._jrdd_val = None
+ self._jrdd_deserializer = self.ctx.serializer
self._bypass_serializer = False
@property
def _jrdd(self):
if self._jrdd_val:
return self._jrdd_val
- func = self.func
- if not self._bypass_serializer and self.ctx.batchSize != 1:
- oldfunc = self.func
- batchSize = self.ctx.batchSize
- def batched_func(split, iterator):
- return batched(oldfunc(split, iterator), batchSize)
- func = batched_func
- cmds = [func, self._bypass_serializer]
- pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds)
+ if self._bypass_serializer:
+ serializer = NoOpSerializer()
+ else:
+ serializer = self.ctx.serializer
+ command = (self.func, self._prev_jrdd_deserializer, serializer)
+ pickled_command = CloudPickleSerializer().dumps(command)
broadcast_vars = ListConverter().convert(
[x._jbroadcast for x in self.ctx._pickled_broadcast_vars],
self.ctx._gateway._gateway_client)
self.ctx._pickled_broadcast_vars.clear()
- class_manifest = self._prev_jrdd.classManifest()
+ class_tag = self._prev_jrdd.classTag()
env = MapConverter().convert(self.ctx.environment,
self.ctx._gateway._gateway_client)
includes = ListConverter().convert(self.ctx._python_includes,
self.ctx._gateway._gateway_client)
python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(),
- pipe_command, env, includes, self.preservesPartitioning, self.ctx.pythonExec,
- broadcast_vars, self.ctx._javaAccumulator, class_manifest)
+ bytearray(pickled_command), env, includes, self.preservesPartitioning,
+ self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator,
+ class_tag)
self._jrdd_val = python_rdd.asJavaRDD()
return self._jrdd_val
diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
index 54fed1c9c7..2a500ab919 100644
--- a/python/pyspark/serializers.py
+++ b/python/pyspark/serializers.py
@@ -15,45 +15,269 @@
# limitations under the License.
#
-import struct
+"""
+PySpark supports custom serializers for transferring data; this can improve
+performance.
+
+By default, PySpark uses L{PickleSerializer} to serialize objects using Python's
+C{cPickle} serializer, which can serialize nearly any Python object.
+Other serializers, like L{MarshalSerializer}, support fewer datatypes but can be
+faster.
+
+The serializer is chosen when creating L{SparkContext}:
+
+>>> from pyspark.context import SparkContext
+>>> from pyspark.serializers import MarshalSerializer
+>>> sc = SparkContext('local', 'test', serializer=MarshalSerializer())
+>>> sc.parallelize(list(range(1000))).map(lambda x: 2 * x).take(10)
+[0, 2, 4, 6, 8, 10, 12, 14, 16, 18]
+>>> sc.stop()
+
+By default, PySpark serialize objects in batches; the batch size can be
+controlled through SparkContext's C{batchSize} parameter
+(the default size is 1024 objects):
+
+>>> sc = SparkContext('local', 'test', batchSize=2)
+>>> rdd = sc.parallelize(range(16), 4).map(lambda x: x)
+
+Behind the scenes, this creates a JavaRDD with four partitions, each of
+which contains two batches of two objects:
+
+>>> rdd.glom().collect()
+[[0, 1, 2, 3], [4, 5, 6, 7], [8, 9, 10, 11], [12, 13, 14, 15]]
+>>> rdd._jrdd.count()
+8L
+>>> sc.stop()
+
+A batch size of -1 uses an unlimited batch size, and a size of 1 disables
+batching:
+
+>>> sc = SparkContext('local', 'test', batchSize=1)
+>>> rdd = sc.parallelize(range(16), 4).map(lambda x: x)
+>>> rdd.glom().collect()
+[[0, 1, 2, 3], [4, 5, 6, 7], [8, 9, 10, 11], [12, 13, 14, 15]]
+>>> rdd._jrdd.count()
+16L
+"""
+
import cPickle
+from itertools import chain, izip, product
+import marshal
+import struct
+from pyspark import cloudpickle
+
+
+__all__ = ["PickleSerializer", "MarshalSerializer"]
+
+
+class SpecialLengths(object):
+ END_OF_DATA_SECTION = -1
+ PYTHON_EXCEPTION_THROWN = -2
+ TIMING_DATA = -3
+
+
+class Serializer(object):
+
+ def dump_stream(self, iterator, stream):
+ """
+ Serialize an iterator of objects to the output stream.
+ """
+ raise NotImplementedError
+
+ def load_stream(self, stream):
+ """
+ Return an iterator of deserialized objects from the input stream.
+ """
+ raise NotImplementedError
+
+
+ def _load_stream_without_unbatching(self, stream):
+ return self.load_stream(stream)
+
+ # Note: our notion of "equality" is that output generated by
+ # equal serializers can be deserialized using the same serializer.
+
+ # This default implementation handles the simple cases;
+ # subclasses should override __eq__ as appropriate.
+
+ def __eq__(self, other):
+ return isinstance(other, self.__class__)
+
+ def __ne__(self, other):
+ return not self.__eq__(other)
+
+
+class FramedSerializer(Serializer):
+ """
+ Serializer that writes objects as a stream of (length, data) pairs,
+ where C{length} is a 32-bit integer and data is C{length} bytes.
+ """
+
+ def dump_stream(self, iterator, stream):
+ for obj in iterator:
+ self._write_with_length(obj, stream)
+
+ def load_stream(self, stream):
+ while True:
+ try:
+ yield self._read_with_length(stream)
+ except EOFError:
+ return
+
+ def _write_with_length(self, obj, stream):
+ serialized = self.dumps(obj)
+ write_int(len(serialized), stream)
+ stream.write(serialized)
+
+ def _read_with_length(self, stream):
+ length = read_int(stream)
+ obj = stream.read(length)
+ if obj == "":
+ raise EOFError
+ return self.loads(obj)
+
+ def dumps(self, obj):
+ """
+ Serialize an object into a byte array.
+ When batching is used, this will be called with an array of objects.
+ """
+ raise NotImplementedError
+
+ def loads(self, obj):
+ """
+ Deserialize an object from a byte array.
+ """
+ raise NotImplementedError
+
+
+class BatchedSerializer(Serializer):
+ """
+ Serializes a stream of objects in batches by calling its wrapped
+ Serializer with streams of objects.
+ """
+
+ UNLIMITED_BATCH_SIZE = -1
+
+ def __init__(self, serializer, batchSize=UNLIMITED_BATCH_SIZE):
+ self.serializer = serializer
+ self.batchSize = batchSize
+
+ def _batched(self, iterator):
+ if self.batchSize == self.UNLIMITED_BATCH_SIZE:
+ yield list(iterator)
+ else:
+ items = []
+ count = 0
+ for item in iterator:
+ items.append(item)
+ count += 1
+ if count == self.batchSize:
+ yield items
+ items = []
+ count = 0
+ if items:
+ yield items
+
+ def dump_stream(self, iterator, stream):
+ self.serializer.dump_stream(self._batched(iterator), stream)
+
+ def load_stream(self, stream):
+ return chain.from_iterable(self._load_stream_without_unbatching(stream))
+
+ def _load_stream_without_unbatching(self, stream):
+ return self.serializer.load_stream(stream)
+
+ def __eq__(self, other):
+ return isinstance(other, BatchedSerializer) and \
+ other.serializer == self.serializer
+
+ def __str__(self):
+ return "BatchedSerializer<%s>" % str(self.serializer)
-class Batch(object):
+class CartesianDeserializer(FramedSerializer):
"""
- Used to store multiple RDD entries as a single Java object.
+ Deserializes the JavaRDD cartesian() of two PythonRDDs.
+ """
+
+ def __init__(self, key_ser, val_ser):
+ self.key_ser = key_ser
+ self.val_ser = val_ser
+
+ def load_stream(self, stream):
+ key_stream = self.key_ser._load_stream_without_unbatching(stream)
+ val_stream = self.val_ser._load_stream_without_unbatching(stream)
+ key_is_batched = isinstance(self.key_ser, BatchedSerializer)
+ val_is_batched = isinstance(self.val_ser, BatchedSerializer)
+ for (keys, vals) in izip(key_stream, val_stream):
+ keys = keys if key_is_batched else [keys]
+ vals = vals if val_is_batched else [vals]
+ for pair in product(keys, vals):
+ yield pair
+
+ def __eq__(self, other):
+ return isinstance(other, CartesianDeserializer) and \
+ self.key_ser == other.key_ser and self.val_ser == other.val_ser
+
+ def __str__(self):
+ return "CartesianDeserializer<%s, %s>" % \
+ (str(self.key_ser), str(self.val_ser))
- This relieves us from having to explicitly track whether an RDD
- is stored as batches of objects and avoids problems when processing
- the union() of batched and unbatched RDDs (e.g. the union() of textFile()
- with another RDD).
+
+class NoOpSerializer(FramedSerializer):
+
+ def loads(self, obj): return obj
+ def dumps(self, obj): return obj
+
+
+class PickleSerializer(FramedSerializer):
"""
- def __init__(self, items):
- self.items = items
+ Serializes objects using Python's cPickle serializer:
+ http://docs.python.org/2/library/pickle.html
-def batched(iterator, batchSize):
- if batchSize == -1: # unlimited batch size
- yield Batch(list(iterator))
- else:
- items = []
- count = 0
- for item in iterator:
- items.append(item)
- count += 1
- if count == batchSize:
- yield Batch(items)
- items = []
- count = 0
- if items:
- yield Batch(items)
+ This serializer supports nearly any Python object, but may
+ not be as fast as more specialized serializers.
+ """
+ def dumps(self, obj): return cPickle.dumps(obj, 2)
+ loads = cPickle.loads
-def dump_pickle(obj):
- return cPickle.dumps(obj, 2)
+class CloudPickleSerializer(PickleSerializer):
+ def dumps(self, obj): return cloudpickle.dumps(obj, 2)
-load_pickle = cPickle.loads
+
+class MarshalSerializer(FramedSerializer):
+ """
+ Serializes objects using Python's Marshal serializer:
+
+ http://docs.python.org/2/library/marshal.html
+
+ This serializer is faster than PickleSerializer but supports fewer datatypes.
+ """
+
+ dumps = marshal.dumps
+ loads = marshal.loads
+
+
+class MUTF8Deserializer(Serializer):
+ """
+ Deserializes streams written by Java's DataOutputStream.writeUTF().
+ """
+
+ def loads(self, stream):
+ length = struct.unpack('>H', stream.read(2))[0]
+ return stream.read(length).decode('utf8')
+
+ def load_stream(self, stream):
+ while True:
+ try:
+ yield self.loads(stream)
+ except struct.error:
+ return
+ except EOFError:
+ return
def read_long(stream):
@@ -85,24 +309,3 @@ def write_int(value, stream):
def write_with_length(obj, stream):
write_int(len(obj), stream)
stream.write(obj)
-
-
-def read_with_length(stream):
- length = read_int(stream)
- obj = stream.read(length)
- if obj == "":
- raise EOFError
- return obj
-
-
-def read_from_pickle_file(stream):
- try:
- while True:
- obj = load_pickle(read_with_length(stream))
- if type(obj) == Batch: # We don't care about inheritance
- for item in obj.items:
- yield item
- else:
- yield obj
- except EOFError:
- return
diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py
index a475959090..1602227a27 100644
--- a/python/pyspark/shell.py
+++ b/python/pyspark/shell.py
@@ -42,12 +42,12 @@ print "Using Python version %s (%s, %s)" % (
platform.python_version(),
platform.python_build()[0],
platform.python_build()[1])
-print "Spark context avaiable as sc."
+print "Spark context available as sc."
if add_files != None:
print "Adding files: [%s]" % ", ".join(add_files)
-# The ./pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
+# The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP,
# which allows us to execute the user's PYTHONSTARTUP file:
_pythonstartup = os.environ.get('OLD_PYTHONSTARTUP')
if _pythonstartup and os.path.isfile(_pythonstartup):
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 29d6a128f6..7acb6eaf10 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -19,6 +19,8 @@
Unit tests for PySpark; additional tests are implemented as doctests in
individual modules.
"""
+from fileinput import input
+from glob import glob
import os
import shutil
import sys
@@ -71,8 +73,8 @@ class TestCheckpoint(PySparkTestCase):
time.sleep(1) # 1 second
self.assertTrue(flatMappedRDD.isCheckpointed())
self.assertEqual(flatMappedRDD.collect(), result)
- self.assertEqual(self.checkpointDir.name,
- os.path.dirname(flatMappedRDD.getCheckpointFile()))
+ self.assertEqual("file:" + self.checkpointDir.name,
+ os.path.dirname(os.path.dirname(flatMappedRDD.getCheckpointFile())))
def test_checkpoint_and_restore(self):
parCollection = self.sc.parallelize([1, 2, 3, 4])
@@ -86,7 +88,8 @@ class TestCheckpoint(PySparkTestCase):
time.sleep(1) # 1 second
self.assertTrue(flatMappedRDD.getCheckpointFile() is not None)
- recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile())
+ recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(),
+ flatMappedRDD._jrdd_deserializer)
self.assertEquals([1, 2, 3, 4], recovered.collect())
@@ -137,6 +140,19 @@ class TestAddFile(PySparkTestCase):
self.assertEqual("Hello World from inside a package!", UserClass().hello())
+class TestRDDFunctions(PySparkTestCase):
+
+ def test_save_as_textfile_with_unicode(self):
+ # Regression test for SPARK-970
+ x = u"\u00A1Hola, mundo!"
+ data = self.sc.parallelize([x])
+ tempFile = NamedTemporaryFile(delete=True)
+ tempFile.close()
+ data.saveAsTextFile(tempFile.name)
+ raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*")))
+ self.assertEqual(x, unicode(raw_contents.strip(), "utf-8"))
+
+
class TestIO(PySparkTestCase):
def test_stdout_redirection(self):
diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py
index d63c2aaef7..f2b3f3c142 100644
--- a/python/pyspark/worker.py
+++ b/python/pyspark/worker.py
@@ -23,23 +23,22 @@ import sys
import time
import socket
import traceback
-from base64 import standard_b64decode
# CloudPickler needs to be imported so that depicklers are registered using the
# copy_reg module.
from pyspark.accumulators import _accumulatorRegistry
from pyspark.broadcast import Broadcast, _broadcastRegistry
from pyspark.cloudpickle import CloudPickler
from pyspark.files import SparkFiles
-from pyspark.serializers import write_with_length, read_with_length, write_int, \
- read_long, write_long, read_int, dump_pickle, load_pickle, read_from_pickle_file
+from pyspark.serializers import write_with_length, write_int, read_long, \
+ write_long, read_int, SpecialLengths, MUTF8Deserializer, PickleSerializer
-def load_obj(infile):
- return load_pickle(standard_b64decode(infile.readline().strip()))
+pickleSer = PickleSerializer()
+mutf8_deserializer = MUTF8Deserializer()
def report_times(outfile, boot, init, finish):
- write_int(-3, outfile)
+ write_int(SpecialLengths.TIMING_DATA, outfile)
write_long(1000 * boot, outfile)
write_long(1000 * init, outfile)
write_long(1000 * finish, outfile)
@@ -52,7 +51,7 @@ def main(infile, outfile):
return
# fetch name of workdir
- spark_files_dir = load_pickle(read_with_length(infile))
+ spark_files_dir = mutf8_deserializer.loads(infile)
SparkFiles._root_directory = spark_files_dir
SparkFiles._is_running_on_worker = True
@@ -60,38 +59,33 @@ def main(infile, outfile):
num_broadcast_variables = read_int(infile)
for _ in range(num_broadcast_variables):
bid = read_long(infile)
- value = read_with_length(infile)
- _broadcastRegistry[bid] = Broadcast(bid, load_pickle(value))
+ value = pickleSer._read_with_length(infile)
+ _broadcastRegistry[bid] = Broadcast(bid, value)
# fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH
sys.path.append(spark_files_dir) # *.py files that were added will be copied here
num_python_includes = read_int(infile)
for _ in range(num_python_includes):
- sys.path.append(os.path.join(spark_files_dir, load_pickle(read_with_length(infile))))
+ filename = mutf8_deserializer.loads(infile)
+ sys.path.append(os.path.join(spark_files_dir, filename))
- # now load function
- func = load_obj(infile)
- bypassSerializer = load_obj(infile)
- if bypassSerializer:
- dumps = lambda x: x
- else:
- dumps = dump_pickle
+ command = pickleSer._read_with_length(infile)
+ (func, deserializer, serializer) = command
init_time = time.time()
- iterator = read_from_pickle_file(infile)
try:
- for obj in func(split_index, iterator):
- write_with_length(dumps(obj), outfile)
+ iterator = deserializer.load_stream(infile)
+ serializer.dump_stream(func(split_index, iterator), outfile)
except Exception as e:
- write_int(-2, outfile)
+ write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile)
write_with_length(traceback.format_exc(), outfile)
sys.exit(-1)
finish_time = time.time()
report_times(outfile, boot_time, init_time, finish_time)
# Mark the beginning of the accumulators section of the output
- write_int(-1, outfile)
- for aid, accum in _accumulatorRegistry.items():
- write_with_length(dump_pickle((aid, accum._value)), outfile)
- write_int(-1, outfile)
+ write_int(SpecialLengths.END_OF_DATA_SECTION, outfile)
+ write_int(len(_accumulatorRegistry), outfile)
+ for (aid, accum) in _accumulatorRegistry.items():
+ pickleSer._write_with_length((aid, accum._value), outfile)
if __name__ == '__main__':
diff --git a/python/run-tests b/python/run-tests
index cbc554ea9d..feba97cee0 100755
--- a/python/run-tests
+++ b/python/run-tests
@@ -29,14 +29,16 @@ FAILED=0
rm -f unit-tests.log
function run_test() {
- $FWDIR/pyspark $1 2>&1 | tee -a unit-tests.log
+ SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log
FAILED=$((PIPESTATUS[0]||$FAILED))
}
run_test "pyspark/rdd.py"
run_test "pyspark/context.py"
+run_test "pyspark/conf.py"
run_test "-m doctest pyspark/broadcast.py"
run_test "-m doctest pyspark/accumulators.py"
+run_test "-m doctest pyspark/serializers.py"
run_test "pyspark/tests.py"
if [[ $FAILED != 0 ]]; then
diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py
index 5bb6f5009f..8e4a6292bc 100755
--- a/python/test_support/userlibrary.py
+++ b/python/test_support/userlibrary.py
@@ -1,3 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
"""
Used to test shipping of code depenencies with SparkContext.addPyFile().
"""
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
index f6bf94be6b..869dbdb9b0 100644
--- a/repl-bin/pom.xml
+++ b/repl-bin/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl-bin_2.9.3</artifactId>
+ <artifactId>spark-repl-bin_2.10</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_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl_2.9.3</artifactId>
+ <artifactId>spark-repl_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run
index 8b5d8300f2..3a6f22f41f 100755
--- a/repl-bin/src/deb/bin/run
+++ b/repl-bin/src/deb/bin/run
@@ -17,7 +17,7 @@
# limitations under the License.
#
-SCALA_VERSION=2.9.3
+SCALA_VERSION=2.10
# Figure out where the Scala framework is installed
FWDIR="$(cd `dirname $0`; pwd)"
@@ -48,8 +48,7 @@ fi
export JAVA_OPTS
# Build up classpath
-CLASSPATH="$SPARK_CLASSPATH"
-CLASSPATH+=":$FWDIR/conf"
+CLASSPATH=":$FWDIR/conf"
for jar in `find $FWDIR -name '*jar'`; do
CLASSPATH+=":$jar"
done
diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor
index bcfae22677..052d76fb8d 100755
--- a/repl-bin/src/deb/bin/spark-executor
+++ b/repl-bin/src/deb/bin/spark-executor
@@ -19,4 +19,4 @@
FWDIR="$(cd `dirname $0`; pwd)"
echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/run spark.executor.MesosExecutorBackend
+exec $FWDIR/run org.apache.spark.executor.MesosExecutorBackend
diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell
index ec7e33e1e3..118349d7c3 100755
--- a/repl-bin/src/deb/bin/spark-shell
+++ b/repl-bin/src/deb/bin/spark-shell
@@ -18,4 +18,4 @@
#
FWDIR="$(cd `dirname $0`; pwd)"
-exec $FWDIR/run spark.repl.Main "$@"
+exec $FWDIR/run org.apache.spark.repl.Main "$@"
diff --git a/repl/lib/scala-jline.jar b/repl/lib/scala-jline.jar
deleted file mode 100644
index 2f18c95cdd..0000000000
--- a/repl/lib/scala-jline.jar
+++ /dev/null
Binary files differ
diff --git a/repl/pom.xml b/repl/pom.xml
index 49d86621dd..2dfe7ac900 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-repl_2.9.3</artifactId>
+ <artifactId>spark-repl_2.10</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_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_2.9.3</artifactId>
+ <artifactId>spark-bagel_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-mllib_2.9.3</artifactId>
+ <artifactId>spark-mllib_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>runtime</scope>
</dependency>
@@ -61,10 +61,12 @@
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-compiler</artifactId>
+ <version>${scala.version}</version>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>jline</artifactId>
+ <version>${scala.version}</version>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
@@ -76,18 +78,18 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -125,7 +127,6 @@
<environmentVariables>
<SPARK_HOME>${basedir}/..</SPARK_HOME>
<SPARK_TESTING>1</SPARK_TESTING>
- <SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
</environmentVariables>
</configuration>
</plugin>
diff --git a/repl/src/main/scala/org/apache/spark/repl/Main.scala b/repl/src/main/scala/org/apache/spark/repl/Main.scala
index 17e149f8ab..14b448d076 100644
--- a/repl/src/main/scala/org/apache/spark/repl/Main.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/Main.scala
@@ -20,12 +20,12 @@ package org.apache.spark.repl
import scala.collection.mutable.Set
object Main {
- private var _interp: SparkILoop = null
-
+ private var _interp: SparkILoop = _
+
def interp = _interp
-
+
def interp_=(i: SparkILoop) { _interp = i }
-
+
def main(args: Array[String]) {
_interp = new SparkILoop
_interp.process(args)
diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
index aea08ff81b..05816941b5 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
@@ -15,27 +15,23 @@
* limitations under the License.
*/
-package org.apache.spark.rdd
-
-import org.apache.spark.{Partition, TaskContext}
+package org.apache.spark.repl
+import scala.tools.nsc.{Settings, CompilerCommand}
+import scala.Predef._
/**
- * 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.
+ * Command class enabling Spark-specific command line options (provided by
+ * <i>org.apache.spark.repl.SparkRunnerSettings</i>).
*/
-private[spark]
-class MapPartitionsWithContextRDD[U: ClassManifest, T: ClassManifest](
- prev: RDD[T],
- f: (TaskContext, Iterator[T]) => Iterator[U],
- preservesPartitioning: Boolean
- ) extends RDD[U](prev) {
-
- override def getPartitions: Array[Partition] = firstParent[T].partitions
+class SparkCommandLine(args: List[String], override val settings: Settings)
+ extends CompilerCommand(args, settings) {
- override val partitioner = if (preservesPartitioning) prev.partitioner else None
+ def this(args: List[String], error: String => Unit) {
+ this(args, new SparkRunnerSettings(error))
+ }
- override def compute(split: Partition, context: TaskContext) =
- f(context, firstParent[T].iterator(split, context))
+ def this(args: List[String]) {
+ this(args, str => Console.println("Error: " + str))
+ }
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
new file mode 100644
index 0000000000..b2e1df173e
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
@@ -0,0 +1,109 @@
+/* NSC -- new Scala compiler
+ * Copyright 2005-2013 LAMP/EPFL
+ * @author Paul Phillips
+ */
+
+package org.apache.spark.repl
+
+import scala.tools.nsc._
+import scala.tools.nsc.interpreter._
+
+import scala.reflect.internal.util.BatchSourceFile
+import scala.tools.nsc.ast.parser.Tokens.EOF
+
+import org.apache.spark.Logging
+
+trait SparkExprTyper extends Logging {
+ val repl: SparkIMain
+
+ import repl._
+ import global.{ reporter => _, Import => _, _ }
+ import definitions._
+ import syntaxAnalyzer.{ UnitParser, UnitScanner, token2name }
+ import naming.freshInternalVarName
+
+ object codeParser extends { val global: repl.global.type = repl.global } with CodeHandlers[Tree] {
+ def applyRule[T](code: String, rule: UnitParser => T): T = {
+ reporter.reset()
+ val scanner = newUnitParser(code)
+ val result = rule(scanner)
+
+ if (!reporter.hasErrors)
+ scanner.accept(EOF)
+
+ result
+ }
+
+ def defns(code: String) = stmts(code) collect { case x: DefTree => x }
+ def expr(code: String) = applyRule(code, _.expr())
+ def stmts(code: String) = applyRule(code, _.templateStats())
+ def stmt(code: String) = stmts(code).last // guaranteed nonempty
+ }
+
+ /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */
+ def parse(line: String): Option[List[Tree]] = debugging(s"""parse("$line")""") {
+ var isIncomplete = false
+ reporter.withIncompleteHandler((_, _) => isIncomplete = true) {
+ val trees = codeParser.stmts(line)
+ if (reporter.hasErrors) Some(Nil)
+ else if (isIncomplete) None
+ else Some(trees)
+ }
+ }
+ // def parsesAsExpr(line: String) = {
+ // import codeParser._
+ // (opt expr line).isDefined
+ // }
+
+ def symbolOfLine(code: String): Symbol = {
+ def asExpr(): Symbol = {
+ val name = freshInternalVarName()
+ // Typing it with a lazy val would give us the right type, but runs
+ // into compiler bugs with things like existentials, so we compile it
+ // behind a def and strip the NullaryMethodType which wraps the expr.
+ val line = "def " + name + " = {\n" + code + "\n}"
+
+ interpretSynthetic(line) match {
+ case IR.Success =>
+ val sym0 = symbolOfTerm(name)
+ // drop NullaryMethodType
+ val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType)
+ if (sym.info.typeSymbol eq UnitClass) NoSymbol
+ else sym
+ case _ => NoSymbol
+ }
+ }
+ def asDefn(): Symbol = {
+ val old = repl.definedSymbolList.toSet
+
+ interpretSynthetic(code) match {
+ case IR.Success =>
+ repl.definedSymbolList filterNot old match {
+ case Nil => NoSymbol
+ case sym :: Nil => sym
+ case syms => NoSymbol.newOverloaded(NoPrefix, syms)
+ }
+ case _ => NoSymbol
+ }
+ }
+ beQuietDuring(asExpr()) orElse beQuietDuring(asDefn())
+ }
+
+ private var typeOfExpressionDepth = 0
+ def typeOfExpression(expr: String, silent: Boolean = true): Type = {
+ if (typeOfExpressionDepth > 2) {
+ logDebug("Terminating typeOfExpression recursion for expression: " + expr)
+ return NoType
+ }
+ typeOfExpressionDepth += 1
+ // Don't presently have a good way to suppress undesirable success output
+ // while letting errors through, so it is first trying it silently: if there
+ // is an error, and errors are desired, then it re-evaluates non-silently
+ // to induce the error message.
+ try beSilentDuring(symbolOfLine(expr).tpe) match {
+ case NoType if !silent => symbolOfLine(expr).tpe // generate error
+ case tpe => tpe
+ }
+ finally typeOfExpressionDepth -= 1
+ }
+}
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 efdd90c47f..87d94d51be 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -1,28 +1,41 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Alexander Spoon
*/
package org.apache.spark.repl
+
import scala.tools.nsc._
import scala.tools.nsc.interpreter._
+import scala.tools.nsc.interpreter.{ Results => IR }
import Predef.{ println => _, _ }
-import java.io.{ BufferedReader, FileReader, PrintWriter }
+import java.io.{ BufferedReader, FileReader }
+import java.util.concurrent.locks.ReentrantLock
import scala.sys.process.Process
-import session._
-import scala.tools.nsc.interpreter.{ Results => IR }
-import scala.tools.util.{ SignalManager, Signallable, Javap }
+import scala.tools.nsc.interpreter.session._
+import scala.util.Properties.{ jdkHome, javaVersion }
+import scala.tools.util.{ Javap }
import scala.annotation.tailrec
-import scala.util.control.Exception.{ ignoring }
import scala.collection.mutable.ListBuffer
import scala.concurrent.ops
-import util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream }
-import interpreter._
-import io.{ File, Sources }
+import scala.tools.nsc.util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream }
+import scala.tools.nsc.interpreter._
+import scala.tools.nsc.io.{ File, Directory }
+import scala.reflect.NameTransformer._
+import scala.tools.nsc.util.ScalaClassLoader
+import scala.tools.nsc.util.ScalaClassLoader._
+import scala.tools.util._
+import scala.language.{implicitConversions, existentials}
+import scala.reflect.{ClassTag, classTag}
+import scala.tools.reflect.StdRuntimeTags._
+
+import java.lang.{Class => jClass}
+import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse}
import org.apache.spark.Logging
+import org.apache.spark.SparkConf
import org.apache.spark.SparkContext
/** The Scala interactive shell. It provides a read-eval-print loop
@@ -37,45 +50,86 @@ import org.apache.spark.SparkContext
* @author Lex Spoon
* @version 1.2
*/
-class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master: Option[String])
+class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
+ val master: Option[String])
extends AnyRef
with LoopCommands
+ with SparkILoopInit
with Logging
{
- def this(in0: BufferedReader, out: PrintWriter, master: String) = this(Some(in0), out, Some(master))
- def this(in0: BufferedReader, out: PrintWriter) = this(Some(in0), out, None)
- def this() = this(None, new PrintWriter(Console.out, true), None)
+ def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master))
+ def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None)
+ def this() = this(None, new JPrintWriter(Console.out, true), None)
var in: InteractiveReader = _ // the input stream from which commands come
var settings: Settings = _
var intp: SparkIMain = _
- /*
- lazy val power = {
- val g = intp.global
- Power[g.type](this, g)
+ @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp
+ @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i
+
+ /** Having inherited the difficult "var-ness" of the repl instance,
+ * I'm trying to work around it by moving operations into a class from
+ * which it will appear a stable prefix.
+ */
+ private def onIntp[T](f: SparkIMain => T): T = f(intp)
+
+ class IMainOps[T <: SparkIMain](val intp: T) {
+ import intp._
+ import global._
+
+ def printAfterTyper(msg: => String) =
+ intp.reporter printMessage afterTyper(msg)
+
+ /** Strip NullaryMethodType artifacts. */
+ private def replInfo(sym: Symbol) = {
+ sym.info match {
+ case NullaryMethodType(restpe) if sym.isAccessor => restpe
+ case info => info
+ }
+ }
+ def echoTypeStructure(sym: Symbol) =
+ printAfterTyper("" + deconstruct.show(replInfo(sym)))
+
+ def echoTypeSignature(sym: Symbol, verbose: Boolean) = {
+ if (verbose) SparkILoop.this.echo("// Type signature")
+ printAfterTyper("" + replInfo(sym))
+
+ if (verbose) {
+ SparkILoop.this.echo("\n// Internal Type structure")
+ echoTypeStructure(sym)
+ }
+ }
+ }
+ implicit def stabilizeIMain(intp: SparkIMain) = new IMainOps[intp.type](intp)
+
+ /** TODO -
+ * -n normalize
+ * -l label with case class parameter names
+ * -c complete - leave nothing out
+ */
+ private def typeCommandInternal(expr: String, verbose: Boolean): Result = {
+ onIntp { intp =>
+ val sym = intp.symbolOfLine(expr)
+ if (sym.exists) intp.echoTypeSignature(sym, verbose)
+ else ""
+ }
}
- */
- // TODO
- // object opt extends AestheticSettings
- //
- @deprecated("Use `intp` instead.", "2.9.0")
- def interpreter = intp
+ var sparkContext: SparkContext = _
- @deprecated("Use `intp` instead.", "2.9.0")
- def interpreter_= (i: SparkIMain): Unit = intp = i
+ override def echoCommandMessage(msg: String) {
+ intp.reporter printMessage msg
+ }
+ // def isAsync = !settings.Yreplsync.value
+ def isAsync = false
+ // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals])
def history = in.history
/** The context class loader at the time this object was created */
protected val originalClassLoader = Thread.currentThread.getContextClassLoader
- // Install a signal handler so we can be prodded.
- private val signallable =
- /*if (isReplDebug) Signallable("Dump repl state.")(dumpCommand())
- else*/ null
-
// classpath entries added via :cp
var addedClasspath: String = ""
@@ -88,63 +142,40 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
/** Record a command for replay should the user request a :replay */
def addReplay(cmd: String) = replayCommandStack ::= cmd
- /** Try to install sigint handler: ignore failure. Signal handler
- * will interrupt current line execution if any is in progress.
- *
- * Attempting to protect the repl from accidental exit, we only honor
- * a single ctrl-C if the current buffer is empty: otherwise we look
- * for a second one within a short time.
- */
- private def installSigIntHandler() {
- def onExit() {
- Console.println("") // avoiding "shell prompt in middle of line" syndrome
- sys.exit(1)
- }
- ignoring(classOf[Exception]) {
- SignalManager("INT") = {
- if (intp == null)
- onExit()
- else if (intp.lineManager.running)
- intp.lineManager.cancel()
- else if (in.currentLine != "") {
- // non-empty buffer, so make them hit ctrl-C a second time
- SignalManager("INT") = onExit()
- io.timer(5)(installSigIntHandler()) // and restore original handler if they don't
- }
- else onExit()
- }
- }
+ def savingReplayStack[T](body: => T): T = {
+ val saved = replayCommandStack
+ try body
+ finally replayCommandStack = saved
+ }
+ def savingReader[T](body: => T): T = {
+ val saved = in
+ try body
+ finally in = saved
}
+
+ def sparkCleanUp(){
+ echo("Stopping spark context.")
+ intp.beQuietDuring {
+ command("sc.stop()")
+ }
+ }
/** Close the interpreter and set the var to null. */
def closeInterpreter() {
if (intp ne null) {
- intp.close
+ sparkCleanUp()
+ intp.close()
intp = null
- Thread.currentThread.setContextClassLoader(originalClassLoader)
}
}
class SparkILoopInterpreter extends SparkIMain(settings, out) {
+ outer =>
+
override lazy val formatting = new Formatting {
def prompt = SparkILoop.this.prompt
}
- override protected def createLineManager() = new Line.Manager {
- override def onRunaway(line: Line[_]): Unit = {
- val template = """
- |// She's gone rogue, captain! Have to take her out!
- |// Calling Thread.stop on runaway %s with offending code:
- |// scala> %s""".stripMargin
-
- echo(template.format(line.thread, line.code))
- // XXX no way to suppress the deprecation warning
- line.thread.stop()
- in.redrawLine()
- }
- }
- override protected def parentClassLoader = {
- SparkHelper.explicitParentLoader(settings).getOrElse( classOf[SparkILoop].getClassLoader )
- }
+ override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader)
}
/** Create a new interpreter. */
@@ -153,8 +184,6 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
settings.classpath append addedClasspath
intp = new SparkILoopInterpreter
- intp.setContextClassLoader()
- installSigIntHandler()
}
/** print a friendly help message */
@@ -194,25 +223,6 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
}
- /** Print a welcome message */
- def printWelcome() {
- val prop = System.getenv("SPARK_BANNER_TEXT")
- val bannerText =
- if (prop != null) prop else
- """Welcome to
- ____ __
- / __/__ ___ _____/ /__
- _\ \/ _ \/ _ `/ __/ '_/
- /___/ .__/\_,_/_/ /_/\_\ version 0.9.0-SNAPSHOT
- /_/
- """
- echo(bannerText)
- import Properties._
- val welcomeMsg = "Using Scala %s (%s, Java %s)".format(
- versionString, javaVmName, javaVersion)
- echo(welcomeMsg)
- }
-
/** Show the history */
lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") {
override def usage = "[num]"
@@ -233,11 +243,17 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
}
- private def echo(msg: String) = {
+ // When you know you are most likely breaking into the middle
+ // of a line being typed. This softens the blow.
+ protected def echoAndRefresh(msg: String) = {
+ echo("\n" + msg)
+ in.redrawLine()
+ }
+ protected def echo(msg: String) = {
out println msg
out.flush()
}
- private def echoNoNL(msg: String) = {
+ protected def echoNoNL(msg: String) = {
out print msg
out.flush()
}
@@ -258,7 +274,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
import LoopCommand.{ cmd, nullary }
- /** Standard commands **/
+ /** Standard commands */
lazy val standardCommands = List(
cmd("cp", "<path>", "add a jar or directory to the classpath", addClasspath),
cmd("help", "[command]", "print this summary or command-specific help", helpCommand),
@@ -267,52 +283,29 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand),
cmd("implicits", "[-v]", "show the implicits in scope", implicitsCommand),
cmd("javap", "<path|class>", "disassemble a file or class name", javapCommand),
- nullary("keybindings", "show how ctrl-[A-Z] and other keys are bound", keybindingsCommand),
cmd("load", "<path>", "load and interpret a Scala file", loadCommand),
nullary("paste", "enter paste mode: all input up to ctrl-D compiled together", pasteCommand),
- //nullary("power", "enable power user mode", powerCmd),
- nullary("quit", "exit the interpreter", () => Result(false, None)),
+// nullary("power", "enable power user mode", powerCmd),
+ nullary("quit", "exit the repl", () => Result(false, None)),
nullary("replay", "reset execution and replay all previous commands", replay),
+ nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand),
shCommand,
nullary("silent", "disable/enable automatic printing of results", verbosity),
- cmd("type", "<expr>", "display the type of an expression without evaluating it", typeCommand)
+ cmd("type", "[-v] <expr>", "display the type of an expression without evaluating it", typeCommand),
+ nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand)
)
/** Power user commands */
lazy val powerCommands: List[LoopCommand] = List(
- //nullary("dump", "displays a view of the interpreter's internal state", dumpCommand),
- //cmd("phase", "<phase>", "set the implicit phase for power commands", phaseCommand),
- cmd("wrap", "<method>", "name of method to wrap around each repl line", wrapCommand) withLongHelp ("""
- |:wrap
- |:wrap clear
- |:wrap <method>
- |
- |Installs a wrapper around each line entered into the repl.
- |Currently it must be the simple name of an existing method
- |with the specific signature shown in the following example.
- |
- |def timed[T](body: => T): T = {
- | val start = System.nanoTime
- | try body
- | finally println((System.nanoTime - start) + " nanos elapsed.")
- |}
- |:wrap timed
- |
- |If given no argument, :wrap names the wrapper installed.
- |An argument of clear will remove the wrapper if any is active.
- |Note that wrappers do not compose (a new one replaces the old
- |one) and also that the :phase command uses the same machinery,
- |so setting :wrap will clear any :phase setting.
- """.stripMargin.trim)
+ // cmd("phase", "<phase>", "set the implicit phase for power commands", phaseCommand)
)
- /*
- private def dumpCommand(): Result = {
- echo("" + power)
- history.asStrings takeRight 30 foreach echo
- in.redrawLine()
- }
- */
+ // private def dumpCommand(): Result = {
+ // echo("" + power)
+ // history.asStrings takeRight 30 foreach echo
+ // in.redrawLine()
+ // }
+ // private def valsCommand(): Result = power.valsDescription
private val typeTransforms = List(
"scala.collection.immutable." -> "immutable.",
@@ -347,10 +340,9 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
}
- private def implicitsCommand(line: String): Result = {
- val intp = SparkILoop.this.intp
+ private def implicitsCommand(line: String): Result = onIntp { intp =>
import intp._
- import global.Symbol
+ import global._
def p(x: Any) = intp.reporter.printMessage("" + x)
@@ -374,7 +366,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
// This groups the members by where the symbol is defined
val byOwner = syms groupBy (_.owner)
- val sortedOwners = byOwner.toList sortBy { case (owner, _) => intp.afterTyper(source.info.baseClasses indexOf owner) }
+ val sortedOwners = byOwner.toList sortBy { case (owner, _) => afterTyper(source.info.baseClasses indexOf owner) }
sortedOwners foreach {
case (owner, members) =>
@@ -405,157 +397,181 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
}
- protected def newJavap() = new Javap(intp.classLoader, new SparkIMain.ReplStrippingWriter(intp)) {
+ private def findToolsJar() = {
+ val jdkPath = Directory(jdkHome)
+ val jar = jdkPath / "lib" / "tools.jar" toFile;
+
+ if (jar isFile)
+ Some(jar)
+ else if (jdkPath.isDirectory)
+ jdkPath.deepFiles find (_.name == "tools.jar")
+ else None
+ }
+ private def addToolsJarToLoader() = {
+ val cl = findToolsJar match {
+ case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader)
+ case _ => intp.classLoader
+ }
+ if (Javap.isAvailable(cl)) {
+ logDebug(":javap available.")
+ cl
+ }
+ else {
+ logDebug(":javap unavailable: no tools.jar at " + jdkHome)
+ intp.classLoader
+ }
+ }
+
+ protected def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) {
override def tryClass(path: String): Array[Byte] = {
- // Look for Foo first, then Foo$, but if Foo$ is given explicitly,
- // we have to drop the $ to find object Foo, then tack it back onto
- // the end of the flattened name.
- def className = intp flatName path
- def moduleName = (intp flatName path.stripSuffix("$")) + "$"
-
- val bytes = super.tryClass(className)
- if (bytes.nonEmpty) bytes
- else super.tryClass(moduleName)
+ val hd :: rest = path split '.' toList;
+ // If there are dots in the name, the first segment is the
+ // key to finding it.
+ if (rest.nonEmpty) {
+ intp optFlatName hd match {
+ case Some(flat) =>
+ val clazz = flat :: rest mkString NAME_JOIN_STRING
+ val bytes = super.tryClass(clazz)
+ if (bytes.nonEmpty) bytes
+ else super.tryClass(clazz + MODULE_SUFFIX_STRING)
+ case _ => super.tryClass(path)
+ }
+ }
+ else {
+ // Look for Foo first, then Foo$, but if Foo$ is given explicitly,
+ // we have to drop the $ to find object Foo, then tack it back onto
+ // the end of the flattened name.
+ def className = intp flatName path
+ def moduleName = (intp flatName path.stripSuffix(MODULE_SUFFIX_STRING)) + MODULE_SUFFIX_STRING
+
+ val bytes = super.tryClass(className)
+ if (bytes.nonEmpty) bytes
+ else super.tryClass(moduleName)
+ }
}
}
+ // private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap())
private lazy val javap =
try newJavap()
catch { case _: Exception => null }
- private def typeCommand(line: String): Result = {
- intp.typeOfExpression(line) match {
- case Some(tp) => tp.toString
- case _ => "Failed to determine type."
+ // Still todo: modules.
+ private def typeCommand(line0: String): Result = {
+ line0.trim match {
+ case "" => ":type [-v] <expression>"
+ case s if s startsWith "-v " => typeCommandInternal(s stripPrefix "-v " trim, true)
+ case s => typeCommandInternal(s, false)
}
}
+ private def warningsCommand(): Result = {
+ if (intp.lastWarnings.isEmpty)
+ "Can't find any cached warnings."
+ else
+ intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) }
+ }
+
private def javapCommand(line: String): Result = {
if (javap == null)
- return ":javap unavailable on this platform."
- if (line == "")
- return ":javap [-lcsvp] [path1 path2 ...]"
-
- javap(words(line)) foreach { res =>
- if (res.isError) return "Failed: " + res.value
- else res.show()
- }
- }
- private def keybindingsCommand(): Result = {
- if (in.keyBindings.isEmpty) "Key bindings unavailable."
- else {
- echo("Reading jline properties for default key bindings.")
- echo("Accuracy not guaranteed: treat this as a guideline only.\n")
- in.keyBindings foreach (x => echo ("" + x))
- }
+ ":javap unavailable, no tools.jar at %s. Set JDK_HOME.".format(jdkHome)
+ else if (javaVersion startsWith "1.7")
+ ":javap not yet working with java 1.7"
+ else if (line == "")
+ ":javap [-lcsvp] [path1 path2 ...]"
+ else
+ javap(words(line)) foreach { res =>
+ if (res.isError) return "Failed: " + res.value
+ else res.show()
+ }
}
+
private def wrapCommand(line: String): Result = {
def failMsg = "Argument to :wrap must be the name of a method with signature [T](=> T): T"
- val intp = SparkILoop.this.intp
- val g: intp.global.type = intp.global
- import g._
-
- words(line) match {
- case Nil =>
- intp.executionWrapper match {
- case "" => "No execution wrapper is set."
- case s => "Current execution wrapper: " + s
- }
- case "clear" :: Nil =>
- intp.executionWrapper match {
- case "" => "No execution wrapper is set."
- case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper."
- }
- case wrapper :: Nil =>
- intp.typeOfExpression(wrapper) match {
- case Some(PolyType(List(targ), MethodType(List(arg), restpe))) =>
- intp setExecutionWrapper intp.pathToTerm(wrapper)
- "Set wrapper to '" + wrapper + "'"
- case Some(x) =>
- failMsg + "\nFound: " + x
- case _ =>
- failMsg + "\nFound: <unknown>"
- }
- case _ => failMsg
+ onIntp { intp =>
+ import intp._
+ import global._
+
+ words(line) match {
+ case Nil =>
+ intp.executionWrapper match {
+ case "" => "No execution wrapper is set."
+ case s => "Current execution wrapper: " + s
+ }
+ case "clear" :: Nil =>
+ intp.executionWrapper match {
+ case "" => "No execution wrapper is set."
+ case s => intp.clearExecutionWrapper() ; "Cleared execution wrapper."
+ }
+ case wrapper :: Nil =>
+ intp.typeOfExpression(wrapper) match {
+ case PolyType(List(targ), MethodType(List(arg), restpe)) =>
+ intp setExecutionWrapper intp.pathToTerm(wrapper)
+ "Set wrapper to '" + wrapper + "'"
+ case tp =>
+ failMsg + "\nFound: <unknown>"
+ }
+ case _ => failMsg
+ }
}
}
private def pathToPhaseWrapper = intp.pathToTerm("$r") + ".phased.atCurrent"
- /*
- private def phaseCommand(name: String): Result = {
- // This line crashes us in TreeGen:
- //
- // if (intp.power.phased set name) "..."
- //
- // Exception in thread "main" java.lang.AssertionError: assertion failed: ._7.type
- // at scala.Predef$.assert(Predef.scala:99)
- // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:69)
- // at scala.tools.nsc.ast.TreeGen.mkAttributedQualifier(TreeGen.scala:44)
- // at scala.tools.nsc.ast.TreeGen.mkAttributedRef(TreeGen.scala:101)
- // at scala.tools.nsc.ast.TreeGen.mkAttributedStableRef(TreeGen.scala:143)
- //
- // But it works like so, type annotated.
- val phased: Phased = power.phased
- import phased.NoPhaseName
-
- if (name == "clear") {
- phased.set(NoPhaseName)
- intp.clearExecutionWrapper()
- "Cleared active phase."
- }
- else if (name == "") phased.get match {
- case NoPhaseName => "Usage: :phase <expr> (e.g. typer, erasure.next, erasure+3)"
- case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get)
- }
- else {
- val what = phased.parse(name)
- if (what.isEmpty || !phased.set(what))
- "'" + name + "' does not appear to represent a valid phase."
- else {
- intp.setExecutionWrapper(pathToPhaseWrapper)
- val activeMessage =
- if (what.toString.length == name.length) "" + what
- else "%s (%s)".format(what, name)
-
- "Active phase is now: " + activeMessage
- }
- }
- }
- */
+ // private def phaseCommand(name: String): Result = {
+ // val phased: Phased = power.phased
+ // import phased.NoPhaseName
+
+ // if (name == "clear") {
+ // phased.set(NoPhaseName)
+ // intp.clearExecutionWrapper()
+ // "Cleared active phase."
+ // }
+ // else if (name == "") phased.get match {
+ // case NoPhaseName => "Usage: :phase <expr> (e.g. typer, erasure.next, erasure+3)"
+ // case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get)
+ // }
+ // else {
+ // val what = phased.parse(name)
+ // if (what.isEmpty || !phased.set(what))
+ // "'" + name + "' does not appear to represent a valid phase."
+ // else {
+ // intp.setExecutionWrapper(pathToPhaseWrapper)
+ // val activeMessage =
+ // if (what.toString.length == name.length) "" + what
+ // else "%s (%s)".format(what, name)
+
+ // "Active phase is now: " + activeMessage
+ // }
+ // }
+ // }
/** Available commands */
- def commands: List[LoopCommand] = standardCommands /* ++ (
+ def commands: List[LoopCommand] = standardCommands /*++ (
if (isReplPower) powerCommands else Nil
)*/
val replayQuestionMessage =
- """|The repl compiler has crashed spectacularly. Shall I replay your
- |session? I can re-run all lines except the last one.
+ """|That entry seems to have slain the compiler. Shall I replay
+ |your session? I can re-run each line except the last one.
|[y/n]
""".trim.stripMargin
- private val crashRecovery: PartialFunction[Throwable, Unit] = {
+ private val crashRecovery: PartialFunction[Throwable, Boolean] = {
case ex: Throwable =>
- if (settings.YrichExes.value) {
- val sources = implicitly[Sources]
- echo("\n" + ex.getMessage)
- echo(
- if (isReplDebug) "[searching " + sources.path + " for exception contexts...]"
- else "[searching for exception contexts...]"
- )
- echo(Exceptional(ex).force().context())
- }
- else {
- echo(util.stackTraceString(ex))
- }
+ echo(intp.global.throwableAsString(ex))
+
ex match {
case _: NoSuchMethodError | _: NoClassDefFoundError =>
- echo("Unrecoverable error.")
+ echo("\nUnrecoverable error.")
throw ex
case _ =>
- def fn(): Boolean = in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() })
+ def fn(): Boolean =
+ try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() })
+ catch { case _: RuntimeException => false }
+
if (fn()) replay()
else echo("\nAbandoning crashed session.")
}
+ true
}
/** The main read-eval-print loop for the repl. It calls
@@ -568,46 +584,69 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
in readLine prompt
}
// return false if repl should exit
- def processLine(line: String): Boolean =
+ def processLine(line: String): Boolean = {
+ if (isAsync) {
+ if (!awaitInitialized()) return false
+ runThunks()
+ }
if (line eq null) false // assume null means EOF
else command(line) match {
case Result(false, _) => false
case Result(_, Some(finalLine)) => addReplay(finalLine) ; true
case _ => true
}
-
- while (true) {
- try if (!processLine(readOneLine)) return
- catch crashRecovery
}
+ def innerLoop() {
+ if ( try processLine(readOneLine()) catch crashRecovery )
+ innerLoop()
+ }
+ innerLoop()
}
/** interpret all lines from a specified file */
def interpretAllFrom(file: File) {
- val oldIn = in
- val oldReplay = replayCommandStack
-
- try file applyReader { reader =>
- in = SimpleReader(reader, out, false)
- echo("Loading " + file + "...")
- loop()
- }
- finally {
- in = oldIn
- replayCommandStack = oldReplay
+ savingReader {
+ savingReplayStack {
+ file applyReader { reader =>
+ in = SimpleReader(reader, out, false)
+ echo("Loading " + file + "...")
+ loop()
+ }
+ }
}
}
- /** create a new interpreter and replay all commands so far */
+ /** create a new interpreter and replay the given commands */
def replay() {
- closeInterpreter()
- createInterpreter()
- for (cmd <- replayCommands) {
+ reset()
+ if (replayCommandStack.isEmpty)
+ echo("Nothing to replay.")
+ else for (cmd <- replayCommands) {
echo("Replaying: " + cmd) // flush because maybe cmd will have its own output
command(cmd)
echo("")
}
}
+ def resetCommand() {
+ echo("Resetting repl state.")
+ if (replayCommandStack.nonEmpty) {
+ echo("Forgetting this session history:\n")
+ replayCommands foreach echo
+ echo("")
+ replayCommandStack = Nil
+ }
+ if (intp.namedDefinedTerms.nonEmpty)
+ echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", "))
+ if (intp.definedTypes.nonEmpty)
+ echo("Forgetting defined types: " + intp.definedTypes.mkString(", "))
+
+ reset()
+ }
+
+ def reset() {
+ intp.reset()
+ // unleashAndSetPhase()
+ }
/** fork a shell and run a command */
lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") {
@@ -664,18 +703,31 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
def powerCmd(): Result = {
if (isReplPower) "Already in power mode."
- else enablePowerMode()
+ else enablePowerMode(false)
}
- def enablePowerMode() = {
- //replProps.power setValue true
- //power.unleash()
- //echo(power.banner)
+
+ def enablePowerMode(isDuringInit: Boolean) = {
+ // replProps.power setValue true
+ // unleashAndSetPhase()
+ // asyncEcho(isDuringInit, power.banner)
+ }
+ // private def unleashAndSetPhase() {
+// if (isReplPower) {
+// // power.unleash()
+// // Set the phase to "typer"
+// intp beSilentDuring phaseCommand("typer")
+// }
+// }
+
+ def asyncEcho(async: Boolean, msg: => String) {
+ if (async) asyncMessage(msg)
+ else echo(msg)
}
def verbosity() = {
- val old = intp.printResults
- intp.printResults = !old
- echo("Switched " + (if (old) "off" else "on") + " result printing.")
+ // val old = intp.printResults
+ // intp.printResults = !old
+ // echo("Switched " + (if (old) "off" else "on") + " result printing.")
}
/** Run one command submitted by the user. Two values are returned:
@@ -716,11 +768,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
def transcript(start: String) = {
- // Printing this message doesn't work very well because it's buried in the
- // transcript they just pasted. Todo: a short timer goes off when
- // lines stop coming which tells them to hit ctrl-D.
- //
- // echo("// Detected repl transcript paste: ctrl-D to finish.")
+ echo("\n// Detected repl transcript paste: ctrl-D to finish.\n")
apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim))
}
}
@@ -777,33 +825,17 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") {
interpretStartingWith(intp.mostRecentVar + code)
}
- else {
- def runCompletion = in.completion execute code map (intp bindValue _)
- /** Due to my accidentally letting file completion execution sneak ahead
- * of actual parsing this now operates in such a way that the scala
- * interpretation always wins. However to avoid losing useful file
- * completion I let it fail and then check the others. So if you
- * type /tmp it will echo a failure and then give you a Directory object.
- * It's not pretty: maybe I'll implement the silence bits I need to avoid
- * echoing the failure.
- */
- if (intp isParseable code) {
- val (code, result) = reallyInterpret
- //if (power != null && code == IR.Error)
- // runCompletion
-
- result
- }
- else runCompletion match {
- case Some(_) => None // completion hit: avoid the latent error
- case _ => reallyInterpret._2 // trigger the latent error
- }
+ else if (code.trim startsWith "//") {
+ // line comment, do nothing
+ None
}
+ else
+ reallyInterpret._2
}
// runs :load `file` on any files passed via -i
def loadFiles(settings: Settings) = settings match {
- case settings: GenericRunnerSettings =>
+ case settings: SparkRunnerSettings =>
for (filename <- settings.loadfiles.value) {
val cmd = ":load " + filename
command(cmd)
@@ -820,7 +852,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
def chooseReader(settings: Settings): InteractiveReader = {
if (settings.Xnojline.value || Properties.isEmacsShell)
SimpleReader()
- else try SparkJLineReader(
+ else try new SparkJLineReader(
if (settings.noCompletion.value) NoCompletion
else new SparkJLineCompletion(intp)
)
@@ -831,100 +863,102 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
}
- def initializeSpark() {
- intp.beQuietDuring {
- command("""
- org.apache.spark.repl.Main.interp.out.println("Creating SparkContext...");
- org.apache.spark.repl.Main.interp.out.flush();
- @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext();
- org.apache.spark.repl.Main.interp.out.println("Spark context available as sc.");
- org.apache.spark.repl.Main.interp.out.flush();
- """)
- command("import org.apache.spark.SparkContext._")
- val prop = System.getenv("SPARK_SHELL_INIT_BLOCK")
- if (prop != null) {
- command(prop)
- }
- }
- echo("Type in expressions to have them evaluated.")
- echo("Type :help for more information.")
- }
-
- var sparkContext: SparkContext = null
-
- def createSparkContext(): SparkContext = {
- val uri = System.getenv("SPARK_EXECUTOR_URI")
- if (uri != null) {
- System.setProperty("spark.executor.uri", uri)
- }
- val master = this.master match {
- case Some(m) => m
- case None => {
- val prop = System.getenv("MASTER")
- if (prop != null) prop else "local"
- }
- }
- val jars = Option(System.getenv("ADD_JARS")).map(_.split(','))
- .getOrElse(new Array[String](0))
- .map(new java.io.File(_).getAbsolutePath)
- try {
- sparkContext = new SparkContext(master, "Spark shell", System.getenv("SPARK_HOME"), jars)
- } catch {
- case e: Exception =>
- e.printStackTrace()
- echo("Failed to create SparkContext, exiting...")
- sys.exit(1)
- }
- sparkContext
- }
-
- def process(settings: Settings): Boolean = {
- // Ensure logging is initialized before any Spark threads try to use logs
- // (because SLF4J initialization is not thread safe)
- initLogging()
-
- printWelcome()
- echo("Initializing interpreter...")
-
- // Add JARS specified in Spark's ADD_JARS variable to classpath
- val jars = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0))
- jars.foreach(settings.classpath.append(_))
+ val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe
+ val m = u.runtimeMirror(getClass.getClassLoader)
+ private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] =
+ u.TypeTag[T](
+ m,
+ new TypeCreator {
+ def apply[U <: ApiUniverse with Singleton](m: Mirror[U]): U # Type =
+ m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type]
+ })
+ def process(settings: Settings): Boolean = savingContextLoader {
this.settings = settings
createInterpreter()
// sets in to some kind of reader depending on environmental cues
in = in0 match {
case Some(reader) => SimpleReader(reader, out, true)
- case None => chooseReader(settings)
+ case None =>
+ // some post-initialization
+ chooseReader(settings) match {
+ case x: SparkJLineReader => addThunk(x.consoleReader.postInit) ; x
+ case x => x
+ }
}
+ lazy val tagOfSparkIMain = tagOfStaticClass[org.apache.spark.repl.SparkIMain]
+ // Bind intp somewhere out of the regular namespace where
+ // we can get at it in generated code.
+ addThunk(intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfSparkIMain, classTag[SparkIMain])))
+ addThunk({
+ import scala.tools.nsc.io._
+ import Properties.userHome
+ import scala.compat.Platform.EOL
+ val autorun = replProps.replAutorunCode.option flatMap (f => io.File(f).safeSlurp())
+ if (autorun.isDefined) intp.quietRun(autorun.get)
+ })
+
+ addThunk(printWelcome())
+ addThunk(initializeSpark())
- loadFiles(settings)
// it is broken on startup; go ahead and exit
if (intp.reporter.hasErrors)
return false
- try {
- // this is about the illusion of snappiness. We call initialize()
- // which spins off a separate thread, then print the prompt and try
- // our best to look ready. Ideally the user will spend a
- // couple seconds saying "wow, it starts so fast!" and by the time
- // they type a command the compiler is ready to roll.
- intp.initialize()
- initializeSpark()
- if (isReplPower) {
- echo("Starting in power mode, one moment...\n")
- enablePowerMode()
- }
- loop()
+ // This is about the illusion of snappiness. We call initialize()
+ // which spins off a separate thread, then print the prompt and try
+ // our best to look ready. The interlocking lazy vals tend to
+ // inter-deadlock, so we break the cycle with a single asynchronous
+ // message to an actor.
+ if (isAsync) {
+ intp initialize initializedCallback()
+ createAsyncListener() // listens for signal to run postInitialization
}
+ else {
+ intp.initializeSynchronous()
+ postInitialization()
+ }
+ // printWelcome()
+
+ loadFiles(settings)
+
+ try loop()
+ catch AbstractOrMissingHandler()
finally closeInterpreter()
+
true
}
+ def createSparkContext(): SparkContext = {
+ val execUri = System.getenv("SPARK_EXECUTOR_URI")
+ val master = this.master match {
+ case Some(m) => m
+ case None => {
+ val prop = System.getenv("MASTER")
+ if (prop != null) prop else "local"
+ }
+ }
+ val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath)
+ val conf = new SparkConf()
+ .setMaster(master)
+ .setAppName("Spark shell")
+ .setJars(jars)
+ .set("spark.repl.class.uri", intp.classServer.uri)
+ if (execUri != null) {
+ conf.set("spark.executor.uri", execUri)
+ }
+ if (System.getenv("SPARK_HOME") != null) {
+ conf.setSparkHome(System.getenv("SPARK_HOME"))
+ }
+ sparkContext = new SparkContext(conf)
+ echo("Created spark context..")
+ sparkContext
+ }
+
/** process command-line arguments and do as they request */
def process(args: Array[String]): Boolean = {
- val command = new CommandLine(args.toList, msg => echo("scala: " + msg))
+ val command = new SparkCommandLine(args.toList, msg => echo(msg))
def neededHelp(): String =
(if (command.settings.help.value) command.usageMsg + "\n" else "") +
(if (command.settings.Xhelp.value) command.xusageMsg + "\n" else "")
@@ -937,13 +971,6 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
}
@deprecated("Use `process` instead", "2.9.0")
- def main(args: Array[String]): Unit = {
- if (isReplDebug)
- System.out.println(new java.util.Date)
-
- process(args)
- }
- @deprecated("Use `process` instead", "2.9.0")
def main(settings: Settings): Unit = process(settings)
}
@@ -951,6 +978,8 @@ object SparkILoop {
implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp
private def echo(msg: String) = Console println msg
+ def getAddedJars: Array[String] = Option(System.getenv("ADD_JARS")).map(_.split(',')).getOrElse(new Array[String](0))
+
// Designed primarily for use by test code: take a String with a
// bunch of code, and prints out a transcript of what it would look
// like if you'd just typed it into the repl.
@@ -959,7 +988,7 @@ object SparkILoop {
stringFromStream { ostream =>
Console.withOut(ostream) {
- val output = new PrintWriter(new OutputStreamWriter(ostream), true) {
+ val output = new JPrintWriter(new OutputStreamWriter(ostream), true) {
override def write(str: String) = {
// completely skip continuation lines
if (str forall (ch => ch.isWhitespace || ch == '|')) ()
@@ -978,9 +1007,12 @@ object SparkILoop {
}
}
val repl = new SparkILoop(input, output)
+
if (settings.classpath.isDefault)
settings.classpath.value = sys.props("java.class.path")
+ getAddedJars.foreach(settings.classpath.append(_))
+
repl process settings
}
}
@@ -995,8 +1027,8 @@ object SparkILoop {
stringFromStream { ostream =>
Console.withOut(ostream) {
val input = new BufferedReader(new StringReader(code))
- val output = new PrintWriter(new OutputStreamWriter(ostream), true)
- val repl = new SparkILoop(input, output)
+ val output = new JPrintWriter(new OutputStreamWriter(ostream), true)
+ val repl = new ILoop(input, output)
if (sets.classpath.isDefault)
sets.classpath.value = sys.props("java.class.path")
@@ -1006,32 +1038,4 @@ object SparkILoop {
}
}
def run(lines: List[String]): String = run(lines map (_ + "\n") mkString)
-
- // provide the enclosing type T
- // in order to set up the interpreter's classpath and parent class loader properly
- def breakIf[T: Manifest](assertion: => Boolean, args: NamedParam*): Unit =
- if (assertion) break[T](args.toList)
-
- // start a repl, binding supplied args
- def break[T: Manifest](args: List[NamedParam]): Unit = {
- val msg = if (args.isEmpty) "" else " Binding " + args.size + " value%s.".format(
- if (args.size == 1) "" else "s"
- )
- echo("Debug repl starting." + msg)
- val repl = new SparkILoop {
- override def prompt = "\ndebug> "
- }
- repl.settings = new Settings(echo)
- repl.settings.embeddedDefaults[T]
- repl.createInterpreter()
- repl.in = SparkJLineReader(repl)
-
- // rebind exit so people don't accidentally call sys.exit by way of predef
- repl.quietRun("""def exit = println("Type :quit to resume program execution.")""")
- args foreach (p => repl.bind(p.name, p.tpe, p.value))
- repl.loop()
-
- echo("\nDebug repl exiting.")
- repl.closeInterpreter()
- }
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
new file mode 100644
index 0000000000..5ddba47789
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
@@ -0,0 +1,151 @@
+/* NSC -- new Scala compiler
+ * Copyright 2005-2013 LAMP/EPFL
+ * @author Paul Phillips
+ */
+
+package org.apache.spark.repl
+
+import scala.tools.nsc._
+import scala.tools.nsc.interpreter._
+
+import scala.reflect.internal.util.Position
+import scala.util.control.Exception.ignoring
+import scala.tools.nsc.util.stackTraceString
+
+/**
+ * Machinery for the asynchronous initialization of the repl.
+ */
+trait SparkILoopInit {
+ self: SparkILoop =>
+
+ /** Print a welcome message */
+ def printWelcome() {
+ val prop = System.getenv("SPARK_BANNER_TEXT")
+ val bannerText =
+ if (prop != null) prop else
+ """Welcome to
+ ____ __
+ / __/__ ___ _____/ /__
+ _\ \/ _ \/ _ `/ __/ '_/
+ /___/ .__/\_,_/_/ /_/\_\ version 0.9.0-SNAPSHOT
+ /_/
+"""
+ echo(bannerText)
+ import Properties._
+ val welcomeMsg = "Using Scala %s (%s, Java %s)".format(
+ versionString, javaVmName, javaVersion)
+ echo(welcomeMsg)
+ echo("Type in expressions to have them evaluated.")
+ echo("Type :help for more information.")
+ }
+
+ protected def asyncMessage(msg: String) {
+ if (isReplInfo || isReplPower)
+ echoAndRefresh(msg)
+ }
+
+ private val initLock = new java.util.concurrent.locks.ReentrantLock()
+ private val initCompilerCondition = initLock.newCondition() // signal the compiler is initialized
+ private val initLoopCondition = initLock.newCondition() // signal the whole repl is initialized
+ private val initStart = System.nanoTime
+
+ private def withLock[T](body: => T): T = {
+ initLock.lock()
+ try body
+ finally initLock.unlock()
+ }
+ // a condition used to ensure serial access to the compiler.
+ @volatile private var initIsComplete = false
+ @volatile private var initError: String = null
+ private def elapsed() = "%.3f".format((System.nanoTime - initStart).toDouble / 1000000000L)
+
+ // the method to be called when the interpreter is initialized.
+ // Very important this method does nothing synchronous (i.e. do
+ // not try to use the interpreter) because until it returns, the
+ // repl's lazy val `global` is still locked.
+ protected def initializedCallback() = withLock(initCompilerCondition.signal())
+
+ // Spins off a thread which awaits a single message once the interpreter
+ // has been initialized.
+ protected def createAsyncListener() = {
+ io.spawn {
+ withLock(initCompilerCondition.await())
+ asyncMessage("[info] compiler init time: " + elapsed() + " s.")
+ postInitialization()
+ }
+ }
+
+ // called from main repl loop
+ protected def awaitInitialized(): Boolean = {
+ if (!initIsComplete)
+ withLock { while (!initIsComplete) initLoopCondition.await() }
+ if (initError != null) {
+ println("""
+ |Failed to initialize the REPL due to an unexpected error.
+ |This is a bug, please, report it along with the error diagnostics printed below.
+ |%s.""".stripMargin.format(initError)
+ )
+ false
+ } else true
+ }
+ // private def warningsThunks = List(
+ // () => intp.bind("lastWarnings", "" + typeTag[List[(Position, String)]], intp.lastWarnings _),
+ // )
+
+ protected def postInitThunks = List[Option[() => Unit]](
+ Some(intp.setContextClassLoader _),
+ if (isReplPower) Some(() => enablePowerMode(true)) else None
+ ).flatten
+ // ++ (
+ // warningsThunks
+ // )
+ // called once after init condition is signalled
+ protected def postInitialization() {
+ try {
+ postInitThunks foreach (f => addThunk(f()))
+ runThunks()
+ } catch {
+ case ex: Throwable =>
+ initError = stackTraceString(ex)
+ throw ex
+ } finally {
+ initIsComplete = true
+
+ if (isAsync) {
+ asyncMessage("[info] total init time: " + elapsed() + " s.")
+ withLock(initLoopCondition.signal())
+ }
+ }
+ }
+
+ def initializeSpark() {
+ intp.beQuietDuring {
+ command("""
+ @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext();
+ """)
+ command("import org.apache.spark.SparkContext._")
+ val prop = System.getenv("SPARK_SHELL_INIT_BLOCK")
+ if (prop != null) {
+ command(prop)
+ }
+ }
+ echo("Spark context available as sc.")
+ }
+
+ // code to be executed only after the interpreter is initialized
+ // and the lazy val `global` can be accessed without risk of deadlock.
+ private var pendingThunks: List[() => Unit] = Nil
+ protected def addThunk(body: => Unit) = synchronized {
+ pendingThunks :+= (() => body)
+ }
+ protected def runThunks(): Unit = synchronized {
+ if (pendingThunks.nonEmpty)
+ logDebug("Clearing " + pendingThunks.size + " thunks.")
+
+ while (pendingThunks.nonEmpty) {
+ val thunk = pendingThunks.head
+ pendingThunks = pendingThunks.tail
+ thunk()
+ }
+ }
+}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
index 870e12de34..59fdb0b37a 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Martin Odersky
*/
@@ -9,304 +9,330 @@ import scala.tools.nsc._
import scala.tools.nsc.interpreter._
import Predef.{ println => _, _ }
-import java.io.{ PrintWriter }
-import java.lang.reflect
+import util.stringFromWriter
+import scala.reflect.internal.util._
import java.net.URL
-import util.{ Set => _, _ }
-import io.{ AbstractFile, PlainFile, VirtualDirectory }
-import reporters.{ ConsoleReporter, Reporter }
-import symtab.{ Flags, Names }
-import scala.tools.nsc.interpreter.{ Results => IR }
+import scala.sys.BooleanProp
+import io.{AbstractFile, PlainFile, VirtualDirectory}
+
+import reporters._
+import symtab.Flags
+import scala.reflect.internal.Names
import scala.tools.util.PathResolver
-import scala.tools.nsc.util.{ ScalaClassLoader, Exceptional }
+import scala.tools.nsc.util.ScalaClassLoader
import ScalaClassLoader.URLClassLoader
-import Exceptional.unwrap
+import scala.tools.nsc.util.Exceptional.unwrap
import scala.collection.{ mutable, immutable }
-import scala.PartialFunction.{ cond, condOpt }
import scala.util.control.Exception.{ ultimately }
-import scala.reflect.NameTransformer
import SparkIMain._
-
-import org.apache.spark.HttpServer
+import java.util.concurrent.Future
+import typechecker.Analyzer
+import scala.language.implicitConversions
+import scala.reflect.runtime.{ universe => ru }
+import scala.reflect.{ ClassTag, classTag }
+import scala.tools.reflect.StdRuntimeTags._
+import scala.util.control.ControlThrowable
+import util.stackTraceString
+
+import org.apache.spark.{HttpServer, SparkConf, Logging}
import org.apache.spark.util.Utils
-import org.apache.spark.SparkEnv
-
-/** An interpreter for Scala code.
- *
- * The main public entry points are compile(), interpret(), and bind().
- * The compile() method loads a complete Scala file. The interpret() method
- * executes one line of Scala code at the request of the user. The bind()
- * method binds an object to a variable that can then be used by later
- * interpreted code.
- *
- * The overall approach is based on compiling the requested code and then
- * using a Java classloader and Java reflection to run the code
- * and access its results.
- *
- * In more detail, a single compiler instance is used
- * to accumulate all successfully compiled or interpreted Scala code. To
- * "interpret" a line of code, the compiler generates a fresh object that
- * includes the line of code and which has public member(s) to export
- * all variables defined by that code. To extract the result of an
- * interpreted line to show the user, a second "result object" is created
- * which imports the variables exported by the above object and then
- * exports a single member named "$export". To accomodate user expressions
- * that read from variables or methods defined in previous statements, "import"
- * statements are used.
- *
- * This interpreter shares the strengths and weaknesses of using the
- * full compiler-to-Java. The main strength is that interpreted code
- * behaves exactly as does compiled code, including running at full speed.
- * The main weakness is that redefining classes and methods is not handled
- * properly, because rebinding at the Java level is technically difficult.
- *
- * @author Moez A. Abdel-Gawad
- * @author Lex Spoon
- */
-class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends SparkImports {
- imain =>
-
- /** construct an interpreter that reports to Console */
- def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true))
- def this() = this(new Settings())
-
- /** whether to print out result lines */
- var printResults: Boolean = true
-
- /** whether to print errors */
- var totalSilence: Boolean = false
-
- private val RESULT_OBJECT_PREFIX = "RequestResult$"
-
- lazy val formatting: Formatting = new Formatting {
- val prompt = Properties.shellPromptString
- }
- import formatting._
-
- val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1")
-
- /** Local directory to save .class files too */
- val outputDir = {
- val tmp = System.getProperty("java.io.tmpdir")
- val rootDir = System.getProperty("spark.repl.classdir", tmp)
- Utils.createTempDir(rootDir)
- }
- if (SPARK_DEBUG_REPL) {
- echo("Output directory: " + outputDir)
- }
- /** Scala compiler virtual directory for outputDir */
- val virtualDirectory = new PlainFile(outputDir)
+// /** directory to save .class files to */
+// private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) {
+// private def pp(root: AbstractFile, indentLevel: Int) {
+// val spaces = " " * indentLevel
+// out.println(spaces + root.name)
+// if (root.isDirectory)
+// root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1))
+// }
+// // print the contents hierarchically
+// def show() = pp(this, 0)
+// }
+
+ /** An interpreter for Scala code.
+ *
+ * The main public entry points are compile(), interpret(), and bind().
+ * The compile() method loads a complete Scala file. The interpret() method
+ * executes one line of Scala code at the request of the user. The bind()
+ * method binds an object to a variable that can then be used by later
+ * interpreted code.
+ *
+ * The overall approach is based on compiling the requested code and then
+ * using a Java classloader and Java reflection to run the code
+ * and access its results.
+ *
+ * In more detail, a single compiler instance is used
+ * to accumulate all successfully compiled or interpreted Scala code. To
+ * "interpret" a line of code, the compiler generates a fresh object that
+ * includes the line of code and which has public member(s) to export
+ * all variables defined by that code. To extract the result of an
+ * interpreted line to show the user, a second "result object" is created
+ * which imports the variables exported by the above object and then
+ * exports members called "$eval" and "$print". To accomodate user expressions
+ * that read from variables or methods defined in previous statements, "import"
+ * statements are used.
+ *
+ * This interpreter shares the strengths and weaknesses of using the
+ * full compiler-to-Java. The main strength is that interpreted code
+ * behaves exactly as does compiled code, including running at full speed.
+ * The main weakness is that redefining classes and methods is not handled
+ * properly, because rebinding at the Java level is technically difficult.
+ *
+ * @author Moez A. Abdel-Gawad
+ * @author Lex Spoon
+ */
+ class SparkIMain(initialSettings: Settings, val out: JPrintWriter) extends SparkImports with Logging {
+ imain =>
- /** Jetty server that will serve our classes to worker nodes */
- val classServer = new HttpServer(outputDir)
+ val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1")
- // Start the classServer and store its URI in a spark system property
- // (which will be passed to executors so that they can connect to it)
- classServer.start()
- System.setProperty("spark.repl.class.uri", classServer.uri)
- if (SPARK_DEBUG_REPL) {
- echo("Class server started, URI = " + classServer.uri)
- }
+ /** Local directory to save .class files too */
+ val outputDir = {
+ val tmp = System.getProperty("java.io.tmpdir")
+ val rootDir = new SparkConf().get("spark.repl.classdir", tmp)
+ Utils.createTempDir(rootDir)
+ }
+ if (SPARK_DEBUG_REPL) {
+ echo("Output directory: " + outputDir)
+ }
- /*
- // directory to save .class files to
- val virtualDirectory = new VirtualDirectory("(memory)", None) {
- private def pp(root: io.AbstractFile, indentLevel: Int) {
- val spaces = " " * indentLevel
- out.println(spaces + root.name)
- if (root.isDirectory)
- root.toList sortBy (_.name) foreach (x => pp(x, indentLevel + 1))
- }
- // print the contents hierarchically
- def show() = pp(this, 0)
- }
- */
-
- /** reporter */
- lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this)
- import reporter.{ printMessage, withoutTruncating }
-
- // not sure if we have some motivation to print directly to console
- private def echo(msg: String) { Console println msg }
-
- // protected def defaultImports: List[String] = List("_root_.scala.sys.exit")
-
- /** We're going to go to some trouble to initialize the compiler asynchronously.
- * It's critical that nothing call into it until it's been initialized or we will
- * run into unrecoverable issues, but the perceived repl startup time goes
- * through the roof if we wait for it. So we initialize it with a future and
- * use a lazy val to ensure that any attempt to use the compiler object waits
- * on the future.
- */
- private val _compiler: Global = newCompiler(settings, reporter)
- private var _initializeComplete = false
- def isInitializeComplete = _initializeComplete
-
- private def _initialize(): Boolean = {
- val source = """
- |class $repl_$init {
- | List(1) map (_ + 1)
- |}
- |""".stripMargin
-
- val result = try {
- new _compiler.Run() compileSources List(new BatchSourceFile("<init>", source))
- if (isReplDebug || settings.debug.value) {
- // Can't use printMessage here, it deadlocks
- Console.println("Repl compiler initialized.")
+ val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles
+ val classServer = new HttpServer(outputDir) /** Jetty server that will serve our classes to worker nodes */
+ private var currentSettings: Settings = initialSettings
+ var printResults = true // whether to print result lines
+ var totalSilence = false // whether to print anything
+ private var _initializeComplete = false // compiler is initialized
+ private var _isInitialized: Future[Boolean] = null // set up initialization future
+ private var bindExceptions = true // whether to bind the lastException variable
+ private var _executionWrapper = "" // code to be wrapped around all lines
+
+
+ // Start the classServer and store its URI in a spark system property
+ // (which will be passed to executors so that they can connect to it)
+ classServer.start()
+ if (SPARK_DEBUG_REPL) {
+ echo("Class server started, URI = " + classServer.uri)
}
- // addImports(defaultImports: _*)
- true
- }
- catch {
- case x: AbstractMethodError =>
- printMessage("""
- |Failed to initialize compiler: abstract method error.
- |This is most often remedied by a full clean and recompile.
- |""".stripMargin
- )
- x.printStackTrace()
- false
- case x: MissingRequirementError => printMessage("""
- |Failed to initialize compiler: %s not found.
- |** Note that as of 2.8 scala does not assume use of the java classpath.
- |** For the old behavior pass -usejavacp to scala, or if using a Settings
- |** object programatically, settings.usejavacp.value = true.""".stripMargin.format(x.req)
+
+ /** We're going to go to some trouble to initialize the compiler asynchronously.
+ * It's critical that nothing call into it until it's been initialized or we will
+ * run into unrecoverable issues, but the perceived repl startup time goes
+ * through the roof if we wait for it. So we initialize it with a future and
+ * use a lazy val to ensure that any attempt to use the compiler object waits
+ * on the future.
+ */
+ private var _classLoader: AbstractFileClassLoader = null // active classloader
+ private val _compiler: Global = newCompiler(settings, reporter) // our private compiler
+
+ private val nextReqId = {
+ var counter = 0
+ () => { counter += 1 ; counter }
+ }
+
+ def compilerClasspath: Seq[URL] = (
+ if (isInitializeComplete) global.classPath.asURLs
+ else new PathResolver(settings).result.asURLs // the compiler's classpath
)
- false
+ def settings = currentSettings
+ def mostRecentLine = prevRequestList match {
+ case Nil => ""
+ case req :: _ => req.originalLine
+ }
+ // Run the code body with the given boolean settings flipped to true.
+ def withoutWarnings[T](body: => T): T = beQuietDuring {
+ val saved = settings.nowarn.value
+ if (!saved)
+ settings.nowarn.value = true
+
+ try body
+ finally if (!saved) settings.nowarn.value = false
}
-
- try result
- finally _initializeComplete = result
- }
-
- // set up initialization future
- private var _isInitialized: () => Boolean = null
- def initialize() = synchronized {
- if (_isInitialized == null)
- _isInitialized = scala.concurrent.ops future _initialize()
- }
- /** the public, go through the future compiler */
- lazy val global: Global = {
- initialize()
+ /** construct an interpreter that reports to Console */
+ def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true))
+ def this() = this(new Settings())
- // blocks until it is ; false means catastrophic failure
- if (_isInitialized()) _compiler
- else null
- }
- @deprecated("Use `global` for access to the compiler instance.", "2.9.0")
- lazy val compiler: global.type = global
-
- import global._
-
- object naming extends {
- val global: imain.global.type = imain.global
- } with Naming {
- // make sure we don't overwrite their unwisely named res3 etc.
- override def freshUserVarName(): String = {
- val name = super.freshUserVarName()
- if (definedNameMap contains name) freshUserVarName()
- else name
+ lazy val repllog: Logger = new Logger {
+ val out: JPrintWriter = imain.out
+ val isInfo: Boolean = BooleanProp keyExists "scala.repl.info"
+ val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug"
+ val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace"
}
- }
- import naming._
-
- // object dossiers extends {
- // val intp: imain.type = imain
- // } with Dossiers { }
- // import dossiers._
-
- lazy val memberHandlers = new {
- val intp: imain.type = imain
- } with SparkMemberHandlers
- import memberHandlers._
-
- def atPickler[T](op: => T): T = atPhase(currentRun.picklerPhase)(op)
- def afterTyper[T](op: => T): T = atPhase(currentRun.typerPhase.next)(op)
-
- /** Temporarily be quiet */
- def beQuietDuring[T](operation: => T): T = {
- val wasPrinting = printResults
- ultimately(printResults = wasPrinting) {
- if (isReplDebug) echo(">> beQuietDuring")
- else printResults = false
-
- operation
+ lazy val formatting: Formatting = new Formatting {
+ val prompt = Properties.shellPromptString
}
- }
- def beSilentDuring[T](operation: => T): T = {
- val saved = totalSilence
- totalSilence = true
- try operation
- finally totalSilence = saved
- }
-
- def quietRun[T](code: String) = beQuietDuring(interpret(code))
-
- /** whether to bind the lastException variable */
- private var bindLastException = true
-
- /** A string representing code to be wrapped around all lines. */
- private var _executionWrapper: String = ""
- def executionWrapper = _executionWrapper
- def setExecutionWrapper(code: String) = _executionWrapper = code
- def clearExecutionWrapper() = _executionWrapper = ""
-
- /** Temporarily stop binding lastException */
- def withoutBindingLastException[T](operation: => T): T = {
- val wasBinding = bindLastException
- ultimately(bindLastException = wasBinding) {
- bindLastException = false
- operation
+ lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this)
+
+ import formatting._
+ import reporter.{ printMessage, withoutTruncating }
+
+ // This exists mostly because using the reporter too early leads to deadlock.
+ private def echo(msg: String) { Console println msg }
+ private def _initSources = List(new BatchSourceFile("<init>", "class $repl_$init { }"))
+ private def _initialize() = {
+ try {
+ // todo. if this crashes, REPL will hang
+ new _compiler.Run() compileSources _initSources
+ _initializeComplete = true
+ true
+ }
+ catch AbstractOrMissingHandler()
}
- }
-
- protected def createLineManager(): Line.Manager = new Line.Manager
- lazy val lineManager = createLineManager()
-
- /** interpreter settings */
- lazy val isettings = new SparkISettings(this)
-
- /** Instantiate a compiler. Subclasses can override this to
- * change the compiler class used by this interpreter. */
- protected def newCompiler(settings: Settings, reporter: Reporter) = {
- settings.outputDirs setSingleOutput virtualDirectory
- settings.exposeEmptyPackage.value = true
- new Global(settings, reporter)
- }
-
- /** the compiler's classpath, as URL's */
- lazy val compilerClasspath: List[URL] = new PathResolver(settings) asURLs
+ private def tquoted(s: String) = "\"\"\"" + s + "\"\"\""
+
+ // argument is a thunk to execute after init is done
+ def initialize(postInitSignal: => Unit) {
+ synchronized {
+ if (_isInitialized == null) {
+ _isInitialized = io.spawn {
+ try _initialize()
+ finally postInitSignal
+ }
+ }
+ }
+ }
+ def initializeSynchronous(): Unit = {
+ if (!isInitializeComplete) {
+ _initialize()
+ assert(global != null, global)
+ }
+ }
+ def isInitializeComplete = _initializeComplete
+
+ /** the public, go through the future compiler */
+ lazy val global: Global = {
+ if (isInitializeComplete) _compiler
+ else {
+ // If init hasn't been called yet you're on your own.
+ if (_isInitialized == null) {
+ logWarning("Warning: compiler accessed before init set up. Assuming no postInit code.")
+ initialize(())
+ }
+ // // blocks until it is ; false means catastrophic failure
+ if (_isInitialized.get()) _compiler
+ else null
+ }
+ }
+ @deprecated("Use `global` for access to the compiler instance.", "2.9.0")
+ lazy val compiler: global.type = global
+
+ import global._
+ import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember}
+ import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass}
+
+ implicit class ReplTypeOps(tp: Type) {
+ def orElse(other: => Type): Type = if (tp ne NoType) tp else other
+ def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp)
+ }
+
+ // TODO: If we try to make naming a lazy val, we run into big time
+ // scalac unhappiness with what look like cycles. It has not been easy to
+ // reduce, but name resolution clearly takes different paths.
+ object naming extends {
+ val global: imain.global.type = imain.global
+ } with Naming {
+ // make sure we don't overwrite their unwisely named res3 etc.
+ def freshUserTermName(): TermName = {
+ val name = newTermName(freshUserVarName())
+ if (definedNameMap contains name) freshUserTermName()
+ else name
+ }
+ def isUserTermName(name: Name) = isUserVarName("" + name)
+ def isInternalTermName(name: Name) = isInternalVarName("" + name)
+ }
+ import naming._
+
+ object deconstruct extends {
+ val global: imain.global.type = imain.global
+ } with StructuredTypeStrings
+
+ lazy val memberHandlers = new {
+ val intp: imain.type = imain
+ } with SparkMemberHandlers
+ import memberHandlers._
+
+ /** Temporarily be quiet */
+ def beQuietDuring[T](body: => T): T = {
+ val saved = printResults
+ printResults = false
+ try body
+ finally printResults = saved
+ }
+ def beSilentDuring[T](operation: => T): T = {
+ val saved = totalSilence
+ totalSilence = true
+ try operation
+ finally totalSilence = saved
+ }
+
+ def quietRun[T](code: String) = beQuietDuring(interpret(code))
+
+
+ private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = {
+ case t: ControlThrowable => throw t
+ case t: Throwable =>
+ logDebug(label + ": " + unwrap(t))
+ logDebug(stackTraceString(unwrap(t)))
+ alt
+ }
+ /** takes AnyRef because it may be binding a Throwable or an Exceptional */
+
+ private def withLastExceptionLock[T](body: => T, alt: => T): T = {
+ assert(bindExceptions, "withLastExceptionLock called incorrectly.")
+ bindExceptions = false
+
+ try beQuietDuring(body)
+ catch logAndDiscard("withLastExceptionLock", alt)
+ finally bindExceptions = true
+ }
+
+ def executionWrapper = _executionWrapper
+ def setExecutionWrapper(code: String) = _executionWrapper = code
+ def clearExecutionWrapper() = _executionWrapper = ""
+
+ /** interpreter settings */
+ lazy val isettings = new SparkISettings(this)
+
+ /** Instantiate a compiler. Overridable. */
+ protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = {
+ settings.outputDirs setSingleOutput virtualDirectory
+ settings.exposeEmptyPackage.value = true
+ new Global(settings, reporter) with ReplGlobal {
+ override def toString: String = "<global>"
+ }
+ }
+
+ /** Parent classloader. Overridable. */
+ protected def parentClassLoader: ClassLoader =
+ SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() )
- /* A single class loader is used for all commands interpreted by this Interpreter.
+ /* A single class loader is used for all commands interpreted by this Interpreter.
It would also be possible to create a new class loader for each command
to interpret. The advantages of the current approach are:
- - Expressions are only evaluated one time. This is especially
- significant for I/O, e.g. "val x = Console.readLine"
-
- The main disadvantage is:
-
- - Objects, classes, and methods cannot be rebound. Instead, definitions
- shadow the old ones, and old code objects refer to the old
- definitions.
- */
- private var _classLoader: AbstractFileClassLoader = null
- def resetClassLoader() = _classLoader = makeClassLoader()
- def classLoader: AbstractFileClassLoader = {
- if (_classLoader == null)
- resetClassLoader()
-
- _classLoader
- }
- private def makeClassLoader(): AbstractFileClassLoader = {
- val parent =
- if (parentClassLoader == null) ScalaClassLoader fromURLs compilerClasspath
- else new URLClassLoader(compilerClasspath, parentClassLoader)
+ - Expressions are only evaluated one time. This is especially
+ significant for I/O, e.g. "val x = Console.readLine"
+
+ The main disadvantage is:
- new AbstractFileClassLoader(virtualDirectory, parent) {
+ - Objects, classes, and methods cannot be rebound. Instead, definitions
+ shadow the old ones, and old code objects refer to the old
+ definitions.
+ */
+ def resetClassLoader() = {
+ logDebug("Setting new classloader: was " + _classLoader)
+ _classLoader = null
+ ensureClassLoader()
+ }
+ final def ensureClassLoader() {
+ if (_classLoader == null)
+ _classLoader = makeClassLoader()
+ }
+ def classLoader: AbstractFileClassLoader = {
+ ensureClassLoader()
+ _classLoader
+ }
+ private class TranslatingClassLoader(parent: ClassLoader) extends AbstractFileClassLoader(virtualDirectory, parent) {
/** Overridden here to try translating a simple name to the generated
* class name if the original attempt fails. This method is used by
* getResourceAsStream as well as findClass.
@@ -314,223 +340,300 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
override protected def findAbstractFile(name: String): AbstractFile = {
super.findAbstractFile(name) match {
// deadlocks on startup if we try to translate names too early
- case null if isInitializeComplete => generatedName(name) map (x => super.findAbstractFile(x)) orNull
- case file => file
+ case null if isInitializeComplete =>
+ generatedName(name) map (x => super.findAbstractFile(x)) orNull
+ case file =>
+ file
}
}
}
- }
- private def loadByName(s: String): JClass =
- (classLoader tryToInitializeClass s) getOrElse sys.error("Failed to load expected class: '" + s + "'")
-
- protected def parentClassLoader: ClassLoader =
- SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() )
-
- def getInterpreterClassLoader() = classLoader
-
- // Set the current Java "context" class loader to this interpreter's class loader
- def setContextClassLoader() = classLoader.setAsContext()
-
- /** Given a simple repl-defined name, returns the real name of
- * the class representing it, e.g. for "Bippy" it may return
- *
- * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy
- */
- def generatedName(simpleName: String): Option[String] = {
- if (simpleName endsWith "$") optFlatName(simpleName.init) map (_ + "$")
- else optFlatName(simpleName)
- }
- def flatName(id: String) = optFlatName(id) getOrElse id
- def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id)
-
- def allDefinedNames = definedNameMap.keys.toList sortBy (_.toString)
- def pathToType(id: String): String = pathToName(newTypeName(id))
- def pathToTerm(id: String): String = pathToName(newTermName(id))
- def pathToName(name: Name): String = {
- if (definedNameMap contains name)
- definedNameMap(name) fullPath name
- else name.toString
- }
+ private def makeClassLoader(): AbstractFileClassLoader =
+ new TranslatingClassLoader(parentClassLoader match {
+ case null => ScalaClassLoader fromURLs compilerClasspath
+ case p => new URLClassLoader(compilerClasspath, p)
+ })
+
+ def getInterpreterClassLoader() = classLoader
+
+ // Set the current Java "context" class loader to this interpreter's class loader
+ def setContextClassLoader() = classLoader.setAsContext()
+
+ /** Given a simple repl-defined name, returns the real name of
+ * the class representing it, e.g. for "Bippy" it may return
+ * {{{
+ * $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy
+ * }}}
+ */
+ def generatedName(simpleName: String): Option[String] = {
+ if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING)
+ else optFlatName(simpleName)
+ }
+ def flatName(id: String) = optFlatName(id) getOrElse id
+ def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id)
+
+ def allDefinedNames = definedNameMap.keys.toList.sorted
+ def pathToType(id: String): String = pathToName(newTypeName(id))
+ def pathToTerm(id: String): String = pathToName(newTermName(id))
+ def pathToName(name: Name): String = {
+ if (definedNameMap contains name)
+ definedNameMap(name) fullPath name
+ else name.toString
+ }
- /** Most recent tree handled which wasn't wholly synthetic. */
- private def mostRecentlyHandledTree: Option[Tree] = {
- prevRequests.reverse foreach { req =>
- req.handlers.reverse foreach {
- case x: MemberDefHandler if x.definesValue && !isInternalVarName(x.name.toString) => return Some(x.member)
- case _ => ()
+ /** Most recent tree handled which wasn't wholly synthetic. */
+ private def mostRecentlyHandledTree: Option[Tree] = {
+ prevRequests.reverse foreach { req =>
+ req.handlers.reverse foreach {
+ case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member)
+ case _ => ()
+ }
}
+ None
}
- None
- }
-
- /** Stubs for work in progress. */
- def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = {
- for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) {
- DBG("Redefining type '%s'\n %s -> %s".format(name, t1, t2))
+
+ /** Stubs for work in progress. */
+ def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = {
+ for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) {
+ logDebug("Redefining type '%s'\n %s -> %s".format(name, t1, t2))
+ }
}
- }
- def handleTermRedefinition(name: TermName, old: Request, req: Request) = {
- for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) {
- // Printing the types here has a tendency to cause assertion errors, like
- // assertion failed: fatal: <refinement> has owner value x, but a class owner is required
- // so DBG is by-name now to keep it in the family. (It also traps the assertion error,
- // but we don't want to unnecessarily risk hosing the compiler's internal state.)
- DBG("Redefining term '%s'\n %s -> %s".format(name, t1, t2))
+ def handleTermRedefinition(name: TermName, old: Request, req: Request) = {
+ for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) {
+ // Printing the types here has a tendency to cause assertion errors, like
+ // assertion failed: fatal: <refinement> has owner value x, but a class owner is required
+ // so DBG is by-name now to keep it in the family. (It also traps the assertion error,
+ // but we don't want to unnecessarily risk hosing the compiler's internal state.)
+ logDebug("Redefining term '%s'\n %s -> %s".format(name, t1, t2))
+ }
}
- }
- def recordRequest(req: Request) {
- if (req == null || referencedNameMap == null)
- return
-
- prevRequests += req
- req.referencedNames foreach (x => referencedNameMap(x) = req)
-
- // warning about serially defining companions. It'd be easy
- // enough to just redefine them together but that may not always
- // be what people want so I'm waiting until I can do it better.
- if (!settings.nowarnings.value) {
+
+ def recordRequest(req: Request) {
+ if (req == null || referencedNameMap == null)
+ return
+
+ prevRequests += req
+ req.referencedNames foreach (x => referencedNameMap(x) = req)
+
+ // warning about serially defining companions. It'd be easy
+ // enough to just redefine them together but that may not always
+ // be what people want so I'm waiting until I can do it better.
for {
name <- req.definedNames filterNot (x => req.definedNames contains x.companionName)
oldReq <- definedNameMap get name.companionName
newSym <- req.definedSymbols get name
oldSym <- oldReq.definedSymbols get name.companionName
+ if Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule }
} {
- printMessage("warning: previously defined %s is not a companion to %s.".format(oldSym, newSym))
- printMessage("Companions must be defined together; you may wish to use :paste mode for this.")
+ afterTyper(replwarn(s"warning: previously defined $oldSym is not a companion to $newSym."))
+ replwarn("Companions must be defined together; you may wish to use :paste mode for this.")
}
- }
-
- // Updating the defined name map
- req.definedNames foreach { name =>
- if (definedNameMap contains name) {
- if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req)
- else handleTermRedefinition(name.toTermName, definedNameMap(name), req)
+
+ // Updating the defined name map
+ req.definedNames foreach { name =>
+ if (definedNameMap contains name) {
+ if (name.isTypeName) handleTypeRedefinition(name.toTypeName, definedNameMap(name), req)
+ else handleTermRedefinition(name.toTermName, definedNameMap(name), req)
+ }
+ definedNameMap(name) = req
}
- definedNameMap(name) = req
}
- }
- /** Parse a line into a sequence of trees. Returns None if the input is incomplete. */
- def parse(line: String): Option[List[Tree]] = {
- var justNeedsMore = false
- reporter.withIncompleteHandler((pos,msg) => {justNeedsMore = true}) {
- // simple parse: just parse it, nothing else
- def simpleParse(code: String): List[Tree] = {
- reporter.reset()
- val unit = new CompilationUnit(new BatchSourceFile("<console>", code))
- val scanner = new syntaxAnalyzer.UnitParser(unit)
-
- scanner.templateStatSeq(false)._2
- }
- val trees = simpleParse(line)
-
- if (reporter.hasErrors) Some(Nil) // the result did not parse, so stop
- else if (justNeedsMore) None
- else Some(trees)
+ def replwarn(msg: => String) {
+ if (!settings.nowarnings.value)
+ printMessage(msg)
}
- }
-
- def isParseable(line: String): Boolean = {
- beSilentDuring {
- parse(line) match {
- case Some(xs) => xs.nonEmpty // parses as-is
- case None => true // incomplete
+
+ def isParseable(line: String): Boolean = {
+ beSilentDuring {
+ try parse(line) match {
+ case Some(xs) => xs.nonEmpty // parses as-is
+ case None => true // incomplete
+ }
+ catch { case x: Exception => // crashed the compiler
+ replwarn("Exception in isParseable(\"" + line + "\"): " + x)
+ false
+ }
}
}
+
+ def compileSourcesKeepingRun(sources: SourceFile*) = {
+ val run = new Run()
+ reporter.reset()
+ run compileSources sources.toList
+ (!reporter.hasErrors, run)
+ }
+
+ /** Compile an nsc SourceFile. Returns true if there are
+ * no compilation errors, or false otherwise.
+ */
+ def compileSources(sources: SourceFile*): Boolean =
+ compileSourcesKeepingRun(sources: _*)._1
+
+ /** Compile a string. Returns true if there are no
+ * compilation errors, or false otherwise.
+ */
+ def compileString(code: String): Boolean =
+ compileSources(new BatchSourceFile("<script>", code))
+
+ /** Build a request from the user. `trees` is `line` after being parsed.
+ */
+ private def buildRequest(line: String, trees: List[Tree]): Request = {
+ executingRequest = new Request(line, trees)
+ executingRequest
+ }
+
+ // rewriting "5 // foo" to "val x = { 5 // foo }" creates broken code because
+ // the close brace is commented out. Strip single-line comments.
+ // ... but for error message output reasons this is not used, and rather than
+ // enclosing in braces it is constructed like "val x =\n5 // foo".
+ private def removeComments(line: String): String = {
+ showCodeIfDebugging(line) // as we're about to lose our // show
+ line.lines map (s => s indexOf "//" match {
+ case -1 => s
+ case idx => s take idx
+ }) mkString "\n"
}
- /** Compile an nsc SourceFile. Returns true if there are
- * no compilation errors, or false otherwise.
- */
- def compileSources(sources: SourceFile*): Boolean = {
- reporter.reset()
- new Run() compileSources sources.toList
- !reporter.hasErrors
+ private def safePos(t: Tree, alt: Int): Int =
+ try t.pos.startOrPoint
+ catch { case _: UnsupportedOperationException => alt }
+
+ // Given an expression like 10 * 10 * 10 we receive the parent tree positioned
+ // at a '*'. So look at each subtree and find the earliest of all positions.
+ private def earliestPosition(tree: Tree): Int = {
+ var pos = Int.MaxValue
+ tree foreach { t =>
+ pos = math.min(pos, safePos(t, Int.MaxValue))
+ }
+ pos
}
- /** Compile a string. Returns true if there are no
- * compilation errors, or false otherwise.
- */
- def compileString(code: String): Boolean =
- compileSources(new BatchSourceFile("<script>", code))
- /** Build a request from the user. `trees` is `line` after being parsed.
- */
- private def buildRequest(line: String, trees: List[Tree]): Request = new Request(line, trees)
-
private def requestFromLine(line: String, synthetic: Boolean): Either[IR.Result, Request] = {
- val trees = parse(indentCode(line)) match {
+ val content = indentCode(line)
+ val trees = parse(content) match {
case None => return Left(IR.Incomplete)
case Some(Nil) => return Left(IR.Error) // parse error or empty input
case Some(trees) => trees
}
-
- // use synthetic vars to avoid filling up the resXX slots
- def varName = if (synthetic) freshInternalVarName() else freshUserVarName()
-
- // Treat a single bare expression specially. This is necessary due to it being hard to
- // modify code at a textual level, and it being hard to submit an AST to the compiler.
- if (trees.size == 1) trees.head match {
- case _:Assign => // we don't want to include assignments
- case _:TermTree | _:Ident | _:Select => // ... but do want these as valdefs.
- requestFromLine("val %s =\n%s".format(varName, line), synthetic) match {
+ logDebug(
+ trees map (t => {
+ // [Eugene to Paul] previously it just said `t map ...`
+ // because there was an implicit conversion from Tree to a list of Trees
+ // however Martin and I have removed the conversion
+ // (it was conflicting with the new reflection API),
+ // so I had to rewrite this a bit
+ val subs = t collect { case sub => sub }
+ subs map (t0 =>
+ " " + safePos(t0, -1) + ": " + t0.shortClass + "\n"
+ ) mkString ""
+ }) mkString "\n"
+ )
+ // If the last tree is a bare expression, pinpoint where it begins using the
+ // AST node position and snap the line off there. Rewrite the code embodied
+ // by the last tree as a ValDef instead, so we can access the value.
+ trees.last match {
+ case _:Assign => // we don't want to include assignments
+ case _:TermTree | _:Ident | _:Select => // ... but do want other unnamed terms.
+ val varName = if (synthetic) freshInternalVarName() else freshUserVarName()
+ val rewrittenLine = (
+ // In theory this would come out the same without the 1-specific test, but
+ // it's a cushion against any more sneaky parse-tree position vs. code mismatches:
+ // this way such issues will only arise on multiple-statement repl input lines,
+ // which most people don't use.
+ if (trees.size == 1) "val " + varName + " =\n" + content
+ else {
+ // The position of the last tree
+ val lastpos0 = earliestPosition(trees.last)
+ // Oh boy, the parser throws away parens so "(2+2)" is mispositioned,
+ // with increasingly hard to decipher positions as we move on to "() => 5",
+ // (x: Int) => x + 1, and more. So I abandon attempts to finesse and just
+ // look for semicolons and newlines, which I'm sure is also buggy.
+ val (raw1, raw2) = content splitAt lastpos0
+ logDebug("[raw] " + raw1 + " <---> " + raw2)
+
+ val adjustment = (raw1.reverse takeWhile (ch => (ch != ';') && (ch != '\n'))).size
+ val lastpos = lastpos0 - adjustment
+
+ // the source code split at the laboriously determined position.
+ val (l1, l2) = content splitAt lastpos
+ logDebug("[adj] " + l1 + " <---> " + l2)
+
+ val prefix = if (l1.trim == "") "" else l1 + ";\n"
+ // Note to self: val source needs to have this precise structure so that
+ // error messages print the user-submitted part without the "val res0 = " part.
+ val combined = prefix + "val " + varName + " =\n" + l2
+
+ logDebug(List(
+ " line" -> line,
+ " content" -> content,
+ " was" -> l2,
+ "combined" -> combined) map {
+ case (label, s) => label + ": '" + s + "'"
+ } mkString "\n"
+ )
+ combined
+ }
+ )
+ // Rewriting "foo ; bar ; 123"
+ // to "foo ; bar ; val resXX = 123"
+ requestFromLine(rewrittenLine, synthetic) match {
case Right(req) => return Right(req withOriginalLine line)
case x => return x
}
- case _ =>
+ case _ =>
}
-
- // figure out what kind of request
Right(buildRequest(line, trees))
}
- /**
- * Interpret one line of input. All feedback, including parse errors
- * and evaluation results, are printed via the supplied compiler's
- * reporter. Values defined are available for future interpreted
- * strings.
- *
- *
- * The return value is whether the line was interpreter successfully,
- * e.g. that there were no parse errors.
- *
+ // normalize non-public types so we don't see protected aliases like Self
+ def normalizeNonPublic(tp: Type) = tp match {
+ case TypeRef(_, sym, _) if sym.isAliasType && !sym.isPublic => tp.dealias
+ case _ => tp
+ }
+
+ /**
+ * Interpret one line of input. All feedback, including parse errors
+ * and evaluation results, are printed via the supplied compiler's
+ * reporter. Values defined are available for future interpreted strings.
*
- * @param line ...
- * @return ...
+ * The return value is whether the line was interpreter successfully,
+ * e.g. that there were no parse errors.
*/
def interpret(line: String): IR.Result = interpret(line, false)
+ def interpretSynthetic(line: String): IR.Result = interpret(line, true)
def interpret(line: String, synthetic: Boolean): IR.Result = {
def loadAndRunReq(req: Request) = {
+ classLoader.setAsContext()
val (result, succeeded) = req.loadAndRun
+
/** To our displeasure, ConsoleReporter offers only printMessage,
* which tacks a newline on the end. Since that breaks all the
* output checking, we have to take one off to balance.
*/
- def show() = {
- if (result == "") ()
- else printMessage(result stripSuffix "\n")
- }
-
if (succeeded) {
- if (printResults)
- show()
+ if (printResults && result != "")
+ printMessage(result stripSuffix "\n")
+ else if (isReplDebug) // show quiet-mode activity
+ printMessage(result.trim.lines map ("[quiet] " + _) mkString "\n")
+
// Book-keeping. Have to record synthetic requests too,
// as they may have been issued for information, e.g. :type
recordRequest(req)
IR.Success
}
- else {
- // don't truncate stack traces
- withoutTruncating(show())
- IR.Error
- }
+ else {
+ // don't truncate stack traces
+ withoutTruncating(printMessage(result))
+ IR.Error
+ }
}
-
+
if (global == null) IR.Error
else requestFromLine(line, synthetic) match {
case Left(result) => result
- case Right(req) =>
+ case Right(req) =>
// null indicates a disallowed statement type; otherwise compile and
// fail if false (implying e.g. a type error)
if (req == null || !req.compile) IR.Error
@@ -546,23 +649,39 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
* @param value the object value to bind to it
* @return an indication of whether the binding succeeded
*/
- def bind(name: String, boundType: String, value: Any): IR.Result = {
+ def bind(name: String, boundType: String, value: Any, modifiers: List[String] = Nil): IR.Result = {
val bindRep = new ReadEvalPrint()
val run = bindRep.compile("""
- |object %s {
- | var value: %s = _
- | def set(x: Any) = value = x.asInstanceOf[%s]
- |}
- """.stripMargin.format(bindRep.evalName, boundType, boundType)
- )
- bindRep.callOpt("set", value) match {
- case Some(_) => interpret("val %s = %s.value".format(name, bindRep.evalPath))
- case _ => DBG("Set failed in bind(%s, %s, %s)".format(name, boundType, value)) ; IR.Error
+ |object %s {
+ | var value: %s = _
+ | def set(x: Any) = value = x.asInstanceOf[%s]
+ |}
+ """.stripMargin.format(bindRep.evalName, boundType, boundType)
+ )
+ bindRep.callEither("set", value) match {
+ case Left(ex) =>
+ logDebug("Set failed in bind(%s, %s, %s)".format(name, boundType, value))
+ logDebug(util.stackTraceString(ex))
+ IR.Error
+
+ case Right(_) =>
+ val line = "%sval %s = %s.value".format(modifiers map (_ + " ") mkString, name, bindRep.evalPath)
+ logDebug("Interpreting: " + line)
+ interpret(line)
}
}
+ def directBind(name: String, boundType: String, value: Any): IR.Result = {
+ val result = bind(name, boundType, value)
+ if (result == IR.Success)
+ directlyBoundNames += newTermName(name)
+ result
+ }
+ def directBind(p: NamedParam): IR.Result = directBind(p.name, p.tpe, p.value)
+ def directBind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = directBind((name, value))
+
def rebind(p: NamedParam): IR.Result = {
val name = p.name
- val oldType = typeOfTerm(name) getOrElse { return IR.Error }
+ val oldType = typeOfTerm(name) orElse { return IR.Error }
val newType = p.tpe
val tempName = freshInternalVarName()
@@ -570,23 +689,27 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
quietRun("val %s = %s.asInstanceOf[%s]".format(name, tempName, newType))
}
def quietImport(ids: String*): IR.Result = beQuietDuring(addImports(ids: _*))
- def addImports(ids: String*): IR.Result =
+ def addImports(ids: String*): IR.Result =
if (ids.isEmpty) IR.Success
else interpret("import " + ids.mkString(", "))
- def quietBind(p: NamedParam): IR.Result = beQuietDuring(bind(p))
- def bind(p: NamedParam): IR.Result = bind(p.name, p.tpe, p.value)
- def bind[T: Manifest](name: String, value: T): IR.Result = bind((name, value))
- def bindValue(x: Any): IR.Result = bind(freshUserVarName(), TypeStrings.fromValue(x), x)
+ def quietBind(p: NamedParam): IR.Result = beQuietDuring(bind(p))
+ def bind(p: NamedParam): IR.Result = bind(p.name, p.tpe, p.value)
+ def bind[T: ru.TypeTag : ClassTag](name: String, value: T): IR.Result = bind((name, value))
+ def bindSyntheticValue(x: Any): IR.Result = bindValue(freshInternalVarName(), x)
+ def bindValue(x: Any): IR.Result = bindValue(freshUserVarName(), x)
+ def bindValue(name: String, x: Any): IR.Result = bind(name, TypeStrings.fromValue(x), x)
/** Reset this interpreter, forgetting all user-specified requests. */
def reset() {
- //virtualDirectory.clear()
- virtualDirectory.delete()
- virtualDirectory.create()
+ clearExecutionWrapper()
resetClassLoader()
resetAllCreators()
prevRequests.clear()
+ referencedNameMap.clear()
+ definedNameMap.clear()
+ virtualDirectory.delete()
+ virtualDirectory.create()
}
/** This instance is no longer needed, so release any resources
@@ -596,9 +719,9 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
reporter.flush()
classServer.stop()
}
-
+
/** Here is where we:
- *
+ *
* 1) Read some source code, and put it in the "read" object.
* 2) Evaluate the read object, and put the result in the "eval" object.
* 3) Create a String for human consumption, and put it in the "print" object.
@@ -608,115 +731,172 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
class ReadEvalPrint(lineId: Int) {
def this() = this(freshLineId())
- val packageName = "$line" + lineId
- val readName = "$read"
- val evalName = "$eval"
- val printName = "$print"
- val valueMethod = "$result" // no-args method giving result
-
+ private var lastRun: Run = _
+ private var evalCaught: Option[Throwable] = None
+ private var conditionalWarnings: List[ConditionalWarning] = Nil
+
+ val packageName = sessionNames.line + lineId
+ val readName = sessionNames.read
+ val evalName = sessionNames.eval
+ val printName = sessionNames.print
+ val resultName = sessionNames.result
+
+ def bindError(t: Throwable) = {
+ if (!bindExceptions) // avoid looping if already binding
+ throw t
+
+ val unwrapped = unwrap(t)
+ withLastExceptionLock[String]({
+ directBind[Throwable]("lastException", unwrapped)(tagOfThrowable, classTag[Throwable])
+ util.stackTraceString(unwrapped)
+ }, util.stackTraceString(unwrapped))
+ }
+
// TODO: split it out into a package object and a regular
// object and we can do that much less wrapping.
def packageDecl = "package " + packageName
-
+
def pathTo(name: String) = packageName + "." + name
def packaged(code: String) = packageDecl + "\n\n" + code
def readPath = pathTo(readName)
def evalPath = pathTo(evalName)
def printPath = pathTo(printName)
-
- def call(name: String, args: Any*): AnyRef =
- evalMethod(name).invoke(evalClass, args.map(_.asInstanceOf[AnyRef]): _*)
-
+
+ def call(name: String, args: Any*): AnyRef = {
+ val m = evalMethod(name)
+ logDebug("Invoking: " + m)
+ if (args.nonEmpty)
+ logDebug(" with args: " + args.mkString(", "))
+
+ m.invoke(evalClass, args.map(_.asInstanceOf[AnyRef]): _*)
+ }
+
+ def callEither(name: String, args: Any*): Either[Throwable, AnyRef] =
+ try Right(call(name, args: _*))
+ catch { case ex: Throwable => Left(ex) }
+
def callOpt(name: String, args: Any*): Option[AnyRef] =
try Some(call(name, args: _*))
- catch { case ex: Exception =>
- quietBind("lastException", ex)
- None
- }
-
- lazy val evalClass = loadByName(evalPath)
- lazy val evalValue = callOpt(valueMethod)
+ catch { case ex: Throwable => bindError(ex) ; None }
- def compile(source: String): Boolean = compileAndSaveRun("<console>", source)
- def lineAfterTyper[T](op: => T): T = {
- assert(lastRun != null, "Internal error: trying to use atPhase, but Run is null." + this)
- atPhase(lastRun.typerPhase.next)(op)
+ class EvalException(msg: String, cause: Throwable) extends RuntimeException(msg, cause) { }
+
+ private def evalError(path: String, ex: Throwable) =
+ throw new EvalException("Failed to load '" + path + "': " + ex.getMessage, ex)
+
+ private def load(path: String): Class[_] = {
+ try Class.forName(path, true, classLoader)
+ catch { case ex: Throwable => evalError(path, unwrap(ex)) }
}
-
+
+ lazy val evalClass = load(evalPath)
+ lazy val evalValue = callEither(resultName) match {
+ case Left(ex) => evalCaught = Some(ex) ; None
+ case Right(result) => Some(result)
+ }
+
+ def compile(source: String): Boolean = compileAndSaveRun("<console>", source)
+
/** The innermost object inside the wrapper, found by
- * following accessPath into the outer one.
- */
+ * following accessPath into the outer one.
+ */
def resolvePathToSymbol(accessPath: String): Symbol = {
- //val readRoot = definitions.getModule(readPath) // the outermost wrapper
- // MATEI: changed this to getClass because the root object is no longer a module (Scala singleton object)
- val readRoot = definitions.getClass(readPath) // the outermost wrapper
- (accessPath split '.').foldLeft(readRoot) { (sym, name) =>
- if (name == "") sym else
- lineAfterTyper(sym.info member newTermName(name))
+ // val readRoot = getRequiredModule(readPath) // the outermost wrapper
+ // MATEI: Changed this to getClass because the root object is no longer a module (Scala singleton object)
+
+ val readRoot = rootMirror.getClassByName(newTypeName(readPath)) // the outermost wrapper
+ (accessPath split '.').foldLeft(readRoot: Symbol) {
+ case (sym, "") => sym
+ case (sym, name) => afterTyper(termMember(sym, name))
}
}
-
- // def compileAndTypeExpr(expr: String): Option[Typer] = {
- // class TyperRun extends Run {
- // override def stopPhase(name: String) = name == "superaccessors"
- // }
- // }
- private var lastRun: Run = _
- private def evalMethod(name: String) = {
- val methods = evalClass.getMethods filter (_.getName == name)
- assert(methods.size == 1, "Internal error - eval object method " + name + " is overloaded: " + methods)
- methods.head
+ /** We get a bunch of repeated warnings for reasons I haven't
+ * entirely figured out yet. For now, squash.
+ */
+ private def updateRecentWarnings(run: Run) {
+ def loop(xs: List[(Position, String)]): List[(Position, String)] = xs match {
+ case Nil => Nil
+ case ((pos, msg)) :: rest =>
+ val filtered = rest filter { case (pos0, msg0) =>
+ (msg != msg0) || (pos.lineContent.trim != pos0.lineContent.trim) || {
+ // same messages and same line content after whitespace removal
+ // but we want to let through multiple warnings on the same line
+ // from the same run. The untrimmed line will be the same since
+ // there's no whitespace indenting blowing it.
+ (pos.lineContent == pos0.lineContent)
+ }
+ }
+ ((pos, msg)) :: loop(filtered)
+ }
+ //PRASHANT: This leads to a NoSuchMethodError for _.warnings. Yet to figure out its purpose.
+ // val warnings = loop(run.allConditionalWarnings flatMap (_.warnings))
+ // if (warnings.nonEmpty)
+ // mostRecentWarnings = warnings
+ }
+ private def evalMethod(name: String) = evalClass.getMethods filter (_.getName == name) match {
+ case Array(method) => method
+ case xs => sys.error("Internal error: eval object " + evalClass + ", " + xs.mkString("\n", "\n", ""))
}
private def compileAndSaveRun(label: String, code: String) = {
showCodeIfDebugging(code)
- reporter.reset()
- lastRun = new Run()
- lastRun.compileSources(List(new BatchSourceFile(label, packaged(code))))
- !reporter.hasErrors
+ val (success, run) = compileSourcesKeepingRun(new BatchSourceFile(label, packaged(code)))
+ updateRecentWarnings(run)
+ lastRun = run
+ success
}
}
/** One line of code submitted by the user for interpretation */
- // private
+ // private
class Request(val line: String, val trees: List[Tree]) {
- val lineRep = new ReadEvalPrint()
- import lineRep.lineAfterTyper
-
+ val reqId = nextReqId()
+ val lineRep = new ReadEvalPrint()
+
private var _originalLine: String = null
def withOriginalLine(s: String): this.type = { _originalLine = s ; this }
def originalLine = if (_originalLine == null) line else _originalLine
/** handlers for each tree in this request */
val handlers: List[MemberHandler] = trees map (memberHandlers chooseHandler _)
+ def defHandlers = handlers collect { case x: MemberDefHandler => x }
/** all (public) names defined by these statements */
val definedNames = handlers flatMap (_.definedNames)
/** list of names used by this expression */
val referencedNames: List[Name] = handlers flatMap (_.referencedNames)
-
+
/** def and val names */
def termNames = handlers flatMap (_.definesTerm)
def typeNames = handlers flatMap (_.definesType)
+ def definedOrImported = handlers flatMap (_.definedOrImported)
+ def definedSymbolList = defHandlers flatMap (_.definedSymbols)
+
+ def definedTypeSymbol(name: String) = definedSymbols(newTypeName(name))
+ def definedTermSymbol(name: String) = definedSymbols(newTermName(name))
/** Code to import bound names from previous lines - accessPath is code to
- * append to objectName to access anything bound by request.
- */
- val ComputedImports(importsPreamble, importsTrailer, accessPath) =
+ * append to objectName to access anything bound by request.
+ */
+ val SparkComputedImports(importsPreamble, importsTrailer, accessPath) =
importsCode(referencedNames.toSet)
/** Code to access a variable with the specified name */
- def fullPath(vname: String) = (
- //lineRep.readPath + accessPath + ".`%s`".format(vname)
+ def fullPath(vname: String) = {
+ // lineRep.readPath + accessPath + ".`%s`".format(vname)
lineRep.readPath + ".INSTANCE" + accessPath + ".`%s`".format(vname)
- )
- /** Same as fullpath, but after it has been flattened, so:
- * $line5.$iw.$iw.$iw.Bippy // fullPath
- * $line5.$iw$$iw$$iw$Bippy // fullFlatName
- */
- def fullFlatName(name: String) =
- lineRep.readPath + accessPath.replace('.', '$') + "$" + name
+ }
+ /** Same as fullpath, but after it has been flattened, so:
+ * $line5.$iw.$iw.$iw.Bippy // fullPath
+ * $line5.$iw$$iw$$iw$Bippy // fullFlatName
+ */
+ def fullFlatName(name: String) =
+ // lineRep.readPath + accessPath.replace('.', '$') + nme.NAME_JOIN_STRING + name
+ lineRep.readPath + ".INSTANCE" + accessPath.replace('.', '$') + nme.NAME_JOIN_STRING + name
+
+ /** The unmangled symbol name, but supplemented with line info. */
+ def disambiguated(name: Name): String = name + " (in " + lineRep + ")"
/** Code to access a variable with the specified name */
def fullPath(vname: Name): String = fullPath(vname.toString)
@@ -726,52 +906,66 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
/** generate the source code for the object that computes this request */
private object ObjectSourceCode extends CodeAssembler[MemberHandler] {
+ def path = pathToTerm("$intp")
+ def envLines = {
+ if (!isReplPower) Nil // power mode only for now
+ // $intp is not bound; punt, but include the line.
+ else if (path == "$intp") List(
+ "def $line = " + tquoted(originalLine),
+ "def $trees = Nil"
+ )
+ else List(
+ "def $line = " + tquoted(originalLine),
+ "def $req = %s.requestForReqId(%s).orNull".format(path, reqId),
+ "def $trees = if ($req eq null) Nil else $req.trees".format(lineRep.readName, path, reqId)
+ )
+ }
+
val preamble = """
|class %s extends Serializable {
- | %s%s
- """.stripMargin.format(lineRep.readName, importsPreamble, indentCode(toCompute))
+ | %s%s%s
+ """.stripMargin.format(lineRep.readName, envLines.map(" " + _ + ";\n").mkString, importsPreamble, indentCode(toCompute))
val postamble = importsTrailer + "\n}" + "\n" +
"object " + lineRep.readName + " {\n" +
" val INSTANCE = new " + lineRep.readName + "();\n" +
"}\n"
val generate = (m: MemberHandler) => m extraCodeToEvaluate Request.this
+
/*
val preamble = """
- |object %s {
- | %s%s
- """.stripMargin.format(lineRep.readName, importsPreamble, indentCode(toCompute))
+ |object %s extends Serializable {
+ |%s%s%s
+ """.stripMargin.format(lineRep.readName, envLines.map(" " + _ + ";\n").mkString, importsPreamble, indentCode(toCompute))
val postamble = importsTrailer + "\n}"
val generate = (m: MemberHandler) => m extraCodeToEvaluate Request.this
*/
+
}
-
+
private object ResultObjectSourceCode extends CodeAssembler[MemberHandler] {
/** We only want to generate this code when the result
* is a value which can be referred to as-is.
- */
+ */
val evalResult =
if (!handlers.last.definesValue) ""
else handlers.last.definesTerm match {
case Some(vname) if typeOf contains vname =>
- """
- |lazy val $result = {
- | $export
- | %s
- |}""".stripMargin.format(fullPath(vname))
+ "lazy val %s = %s".format(lineRep.resultName, fullPath(vname))
case _ => ""
}
+
// first line evaluates object to make sure constructor is run
// initial "" so later code can uniformly be: + etc
val preamble = """
|object %s {
| %s
- | val $export: String = %s {
+ | val %s: String = %s {
| %s
| (""
""".stripMargin.format(
- lineRep.evalName, evalResult, executionWrapper, lineRep.readName + ".INSTANCE" + accessPath
+ lineRep.evalName, evalResult, lineRep.printName,
+ executionWrapper, lineRep.readName + ".INSTANCE" + accessPath
)
-
val postamble = """
| )
| }
@@ -785,7 +979,7 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
def getEval: Option[AnyRef] = {
// ensure it has been compiled
compile
- // try to load it and call the value method
+ // try to load it and call the value method
lineRep.evalValue filterNot (_ == null)
}
@@ -797,121 +991,54 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
// compile the object containing the user's code
lineRep.compile(ObjectSourceCode(handlers)) && {
- // extract and remember types
+ // extract and remember types
typeOf
typesOfDefinedTerms
+ // Assign symbols to the original trees
+ // TODO - just use the new trees.
+ defHandlers foreach { dh =>
+ val name = dh.member.name
+ definedSymbols get name foreach { sym =>
+ dh.member setSymbol sym
+ logDebug("Set symbol of " + name + " to " + sym.defString)
+ }
+ }
+
// compile the result-extraction object
- lineRep compile ResultObjectSourceCode(handlers)
+ withoutWarnings(lineRep compile ResultObjectSourceCode(handlers))
}
}
lazy val resultSymbol = lineRep.resolvePathToSymbol(accessPath)
- def applyToResultMember[T](name: Name, f: Symbol => T) = lineAfterTyper(f(resultSymbol.info.nonPrivateDecl(name)))
+ def applyToResultMember[T](name: Name, f: Symbol => T) = afterTyper(f(resultSymbol.info.nonPrivateDecl(name)))
/* typeOf lookup with encoding */
- def lookupTypeOf(name: Name) = {
- typeOf.getOrElse(name, typeOf(global.encode(name.toString)))
- }
- def simpleNameOfType(name: TypeName) = {
- (compilerTypeOf get name) map (_.typeSymbol.simpleName)
- }
-
- private def typeMap[T](f: Type => T): Map[Name, T] = {
- def toType(name: Name): T = {
- // the types are all =>T; remove the =>
- val tp1 = lineAfterTyper(resultSymbol.info.nonPrivateDecl(name).tpe match {
- case NullaryMethodType(tp) => tp
- case tp => tp
- })
- // normalize non-public types so we don't see protected aliases like Self
- lineAfterTyper(tp1 match {
- case TypeRef(_, sym, _) if !sym.isPublic => f(tp1.normalize)
- case tp => f(tp)
- })
- }
- termNames ++ typeNames map (x => x -> toType(x)) toMap
- }
+ def lookupTypeOf(name: Name) = typeOf.getOrElse(name, typeOf(global.encode(name.toString)))
+ def simpleNameOfType(name: TypeName) = (compilerTypeOf get name) map (_.typeSymbol.simpleName)
+
+ private def typeMap[T](f: Type => T) =
+ mapFrom[Name, Name, T](termNames ++ typeNames)(x => f(cleanMemberDecl(resultSymbol, x)))
+
/** Types of variables defined by this request. */
- lazy val compilerTypeOf = typeMap[Type](x => x)
+ lazy val compilerTypeOf = typeMap[Type](x => x) withDefaultValue NoType
/** String representations of same. */
- lazy val typeOf = typeMap[String](_.toString)
-
+ lazy val typeOf = typeMap[String](tp => afterTyper(tp.toString))
+
// lazy val definedTypes: Map[Name, Type] = {
// typeNames map (x => x -> afterTyper(resultSymbol.info.nonPrivateDecl(x).tpe)) toMap
// }
- lazy val definedSymbols: Map[Name, Symbol] = (
+ lazy val definedSymbols = (
termNames.map(x => x -> applyToResultMember(x, x => x)) ++
- typeNames.map(x => x -> compilerTypeOf.get(x).map(_.typeSymbol).getOrElse(NoSymbol))
- ).toMap
-
- lazy val typesOfDefinedTerms: Map[Name, Type] =
- termNames map (x => x -> applyToResultMember(x, _.tpe)) toMap
-
- private def bindExceptionally(t: Throwable) = {
- val ex: Exceptional =
- if (isettings.showInternalStackTraces) Exceptional(t)
- else new Exceptional(t) {
- override def spanFn(frame: JavaStackFrame) = !(frame.className startsWith lineRep.evalPath)
- override def contextPrelude = super.contextPrelude + "/* The repl internal portion of the stack trace is elided. */\n"
- }
-
- quietBind("lastException", ex)
- ex.contextHead + "\n(access lastException for the full trace)"
- }
- private def bindUnexceptionally(t: Throwable) = {
- quietBind("lastException", t)
- stackTraceString(t)
- }
+ typeNames.map(x => x -> compilerTypeOf(x).typeSymbolDirect)
+ ).toMap[Name, Symbol] withDefaultValue NoSymbol
+
+ lazy val typesOfDefinedTerms = mapFrom[Name, Name, Type](termNames)(x => applyToResultMember(x, _.tpe))
/** load and run the code using reflection */
def loadAndRun: (String, Boolean) = {
- import interpreter.Line._
-
- def handleException(t: Throwable) = {
- /** We turn off the binding to accomodate ticket #2817 */
- withoutBindingLastException {
- val message =
- if (opt.richExes) bindExceptionally(unwrap(t))
- else bindUnexceptionally(unwrap(t))
-
- (message, false)
- }
- }
-
- // Get a copy of the local properties from SparkContext, and set it later in the thread
- // that triggers the execution. This is to make sure the caller of this function can pass
- // the right thread local (inheritable) properties down into Spark.
- val sc = org.apache.spark.repl.Main.interp.sparkContext
- val props = if (sc != null) sc.getLocalProperties() else null
-
- try {
- val execution = lineManager.set(originalLine) {
- // MATEI: set the right SparkEnv for our SparkContext, because
- // this execution will happen in a separate thread
- if (sc != null && sc.env != null) {
- SparkEnv.set(sc.env)
- sc.setLocalProperties(props)
- }
- // Execute the line
- lineRep call "$export"
- }
- execution.await()
-
- execution.state match {
- case Done => ("" + execution.get(), true)
- case Threw =>
- val ex = execution.caught()
- if (isReplDebug)
- ex.printStackTrace()
-
- if (bindLastException) handleException(ex)
- else throw ex
- case Cancelled => ("Execution interrupted by signal.\n", false)
- case Running => ("Execution still running! Seems impossible.", false)
- }
- }
- finally lineManager.clear()
+ try { ("" + (lineRep call sessionNames.print), true) }
+ catch { case ex: Throwable => (lineRep.bindError(ex), false) }
}
override def toString = "Request(line=%s, %s trees)".format(line, trees.size)
@@ -929,136 +1056,157 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
case ModuleDef(_, name, _) => name
case _ => naming.mostRecentVar
})
-
- private def requestForName(name: Name): Option[Request] = {
+
+ private var mostRecentWarnings: List[(global.Position, String)] = Nil
+ def lastWarnings = mostRecentWarnings
+
+ def treesForRequestId(id: Int): List[Tree] =
+ requestForReqId(id).toList flatMap (_.trees)
+
+ def requestForReqId(id: Int): Option[Request] =
+ if (executingRequest != null && executingRequest.reqId == id) Some(executingRequest)
+ else prevRequests find (_.reqId == id)
+
+ def requestForName(name: Name): Option[Request] = {
assert(definedNameMap != null, "definedNameMap is null")
definedNameMap get name
}
- private def requestForIdent(line: String): Option[Request] =
+ def requestForIdent(line: String): Option[Request] =
requestForName(newTermName(line)) orElse requestForName(newTypeName(line))
-
- def safeClass(name: String): Option[Symbol] = {
- try Some(definitions.getClass(newTypeName(name)))
- catch { case _: MissingRequirementError => None }
- }
- def safeModule(name: String): Option[Symbol] = {
- try Some(definitions.getModule(newTermName(name)))
- catch { case _: MissingRequirementError => None }
- }
+
+ def requestHistoryForName(name: Name): List[Request] =
+ prevRequests.toList.reverse filter (_.definedNames contains name)
+
def definitionForName(name: Name): Option[MemberHandler] =
requestForName(name) flatMap { req =>
req.handlers find (_.definedNames contains name)
}
-
+
def valueOfTerm(id: String): Option[AnyRef] =
- requestForIdent(id) flatMap (_.getEval)
+ requestForName(newTermName(id)) flatMap (_.getEval)
def classOfTerm(id: String): Option[JClass] =
- valueOfTerm(id) map (_.getClass)
+ valueOfTerm(id) map (_.getClass)
- def typeOfTerm(id: String): Option[Type] = newTermName(id) match {
- case nme.ROOTPKG => Some(definitions.RootClass.tpe)
- case name => requestForName(name) flatMap (_.compilerTypeOf get name)
+ def typeOfTerm(id: String): Type = newTermName(id) match {
+ case nme.ROOTPKG => RootClass.tpe
+ case name => requestForName(name).fold(NoType: Type)(_ compilerTypeOf name)
}
+
+ def symbolOfType(id: String): Symbol =
+ requestForName(newTypeName(id)).fold(NoSymbol: Symbol)(_ definedTypeSymbol id)
+
def symbolOfTerm(id: String): Symbol =
- requestForIdent(id) flatMap (_.definedSymbols get newTermName(id)) getOrElse NoSymbol
+ requestForIdent(newTermName(id)).fold(NoSymbol: Symbol)(_ definedTermSymbol id)
def runtimeClassAndTypeOfTerm(id: String): Option[(JClass, Type)] = {
- for {
- clazz <- classOfTerm(id)
- tpe <- runtimeTypeOfTerm(id)
- nonAnon <- new RichClass(clazz).supers.find(c => !(new RichClass(c).isScalaAnonymous))
- } yield {
- (nonAnon, tpe)
- }
- }
-
- def runtimeTypeOfTerm(id: String): Option[Type] = {
- for {
- tpe <- typeOfTerm(id)
- clazz <- classOfTerm(id)
- val staticSym = tpe.typeSymbol
- runtimeSym <- safeClass(clazz.getName)
- if runtimeSym != staticSym
- if runtimeSym isSubClass staticSym
- } yield {
- runtimeSym.info
+ classOfTerm(id) flatMap { clazz =>
+ new RichClass(clazz).supers find(c => !(new RichClass(c).isScalaAnonymous)) map { nonAnon =>
+ (nonAnon, runtimeTypeOfTerm(id))
+ }
}
}
-
- // XXX literals.
- // 1) Identifiers defined in the repl.
- // 2) A path loadable via getModule.
- // 3) Try interpreting it as an expression.
- private var typeOfExpressionDepth = 0
- def typeOfExpression(expr: String): Option[Type] = {
- DBG("typeOfExpression(" + expr + ")")
- if (typeOfExpressionDepth > 2) {
- DBG("Terminating typeOfExpression recursion for expression: " + expr)
- return None
- }
- def asQualifiedImport = {
- val name = expr.takeWhile(_ != '.')
- importedTermNamed(name) flatMap { sym =>
- typeOfExpression(sym.fullName + expr.drop(name.length))
- }
+ def runtimeTypeOfTerm(id: String): Type = {
+ typeOfTerm(id) andAlso { tpe =>
+ val clazz = classOfTerm(id) getOrElse { return NoType }
+ val staticSym = tpe.typeSymbol
+ val runtimeSym = getClassIfDefined(clazz.getName)
+
+ if ((runtimeSym != NoSymbol) && (runtimeSym != staticSym) && (runtimeSym isSubClass staticSym))
+ runtimeSym.info
+ else NoType
}
- def asModule = safeModule(expr) map (_.tpe)
- def asExpr = beSilentDuring {
- val lhs = freshInternalVarName()
- val line = "lazy val " + lhs + " = { " + expr + " } "
-
- interpret(line, true) match {
- case IR.Success => typeOfExpression(lhs)
- case _ => None
+ }
+ def cleanMemberDecl(owner: Symbol, member: Name): Type = afterTyper {
+ normalizeNonPublic {
+ owner.info.nonPrivateDecl(member).tpe match {
+ case NullaryMethodType(tp) => tp
+ case tp => tp
}
}
-
- typeOfExpressionDepth += 1
- try typeOfTerm(expr) orElse asModule orElse asExpr orElse asQualifiedImport
- finally typeOfExpressionDepth -= 1
}
- // def compileAndTypeExpr(expr: String): Option[Typer] = {
- // class TyperRun extends Run {
- // override def stopPhase(name: String) = name == "superaccessors"
- // }
- // }
-
+
+ object exprTyper extends {
+ val repl: SparkIMain.this.type = imain
+ } with SparkExprTyper { }
+
+ def parse(line: String): Option[List[Tree]] = exprTyper.parse(line)
+
+ def symbolOfLine(code: String): Symbol =
+ exprTyper.symbolOfLine(code)
+
+ def typeOfExpression(expr: String, silent: Boolean = true): Type =
+ exprTyper.typeOfExpression(expr, silent)
+
protected def onlyTerms(xs: List[Name]) = xs collect { case x: TermName => x }
protected def onlyTypes(xs: List[Name]) = xs collect { case x: TypeName => x }
-
- def definedTerms = onlyTerms(allDefinedNames) filterNot (x => isInternalVarName(x.toString))
- def definedTypes = onlyTypes(allDefinedNames)
- def definedSymbols = prevRequests.toSet flatMap ((x: Request) => x.definedSymbols.values)
-
+
+ def definedTerms = onlyTerms(allDefinedNames) filterNot isInternalTermName
+ def definedTypes = onlyTypes(allDefinedNames)
+ def definedSymbols = prevRequestList.flatMap(_.definedSymbols.values).toSet[Symbol]
+ def definedSymbolList = prevRequestList flatMap (_.definedSymbolList) filterNot (s => isInternalTermName(s.name))
+
+ // Terms with user-given names (i.e. not res0 and not synthetic)
+ def namedDefinedTerms = definedTerms filterNot (x => isUserVarName("" + x) || directlyBoundNames(x))
+
+ private def findName(name: Name) = definedSymbols find (_.name == name) getOrElse NoSymbol
+
+ /** Translate a repl-defined identifier into a Symbol.
+ */
+ def apply(name: String): Symbol =
+ types(name) orElse terms(name)
+
+ def types(name: String): Symbol = {
+ val tpname = newTypeName(name)
+ findName(tpname) orElse getClassIfDefined(tpname)
+ }
+ def terms(name: String): Symbol = {
+ val termname = newTypeName(name)
+ findName(termname) orElse getModuleIfDefined(termname)
+ }
+ // [Eugene to Paul] possibly you could make use of TypeTags here
+ def types[T: ClassTag] : Symbol = types(classTag[T].runtimeClass.getName)
+ def terms[T: ClassTag] : Symbol = terms(classTag[T].runtimeClass.getName)
+ def apply[T: ClassTag] : Symbol = apply(classTag[T].runtimeClass.getName)
+
+ def classSymbols = allDefSymbols collect { case x: ClassSymbol => x }
+ def methodSymbols = allDefSymbols collect { case x: MethodSymbol => x }
+
/** the previous requests this interpreter has processed */
- private lazy val prevRequests = mutable.ListBuffer[Request]()
- private lazy val referencedNameMap = mutable.Map[Name, Request]()
- private lazy val definedNameMap = mutable.Map[Name, Request]()
- protected def prevRequestList = prevRequests.toList
- private def allHandlers = prevRequestList flatMap (_.handlers)
- def allSeenTypes = prevRequestList flatMap (_.typeOf.values.toList) distinct
- def allImplicits = allHandlers filter (_.definesImplicit) flatMap (_.definedNames)
- def importHandlers = allHandlers collect { case x: ImportHandler => x }
-
+ private var executingRequest: Request = _
+ private val prevRequests = mutable.ListBuffer[Request]()
+ private val referencedNameMap = mutable.Map[Name, Request]()
+ private val definedNameMap = mutable.Map[Name, Request]()
+ private val directlyBoundNames = mutable.Set[Name]()
+
+ def allHandlers = prevRequestList flatMap (_.handlers)
+ def allDefHandlers = allHandlers collect { case x: MemberDefHandler => x }
+ def allDefSymbols = allDefHandlers map (_.symbol) filter (_ ne NoSymbol)
+
+ def lastRequest = if (prevRequests.isEmpty) null else prevRequests.last
+ def prevRequestList = prevRequests.toList
+ def allSeenTypes = prevRequestList flatMap (_.typeOf.values.toList) distinct
+ def allImplicits = allHandlers filter (_.definesImplicit) flatMap (_.definedNames)
+ def importHandlers = allHandlers collect { case x: ImportHandler => x }
+
def visibleTermNames: List[Name] = definedTerms ++ importedTerms distinct
/** Another entry point for tab-completion, ids in scope */
def unqualifiedIds = visibleTermNames map (_.toString) filterNot (_ contains "$") sorted
-
+
/** Parse the ScalaSig to find type aliases */
def aliasForType(path: String) = ByteCode.aliasForType(path)
-
+
def withoutUnwrapping(op: => Unit): Unit = {
val saved = isettings.unwrapStrings
isettings.unwrapStrings = false
try op
finally isettings.unwrapStrings = saved
}
-
+
def symbolDefString(sym: Symbol) = {
TypeStrings.quieter(
afterTyper(sym.defString),
@@ -1066,38 +1214,41 @@ class SparkIMain(val settings: Settings, protected val out: PrintWriter) extends
sym.owner.fullName + "."
)
}
-
+
def showCodeIfDebugging(code: String) {
/** Secret bookcase entrance for repl debuggers: end the line
* with "// show" and see what's going on.
*/
- if (SPARK_DEBUG_REPL || code.lines.exists(_.trim endsWith "// show")) {
- echo(code)
- parse(code) foreach (ts => ts foreach (t => withoutUnwrapping(DBG(asCompactString(t)))))
+ def isShow = code.lines exists (_.trim endsWith "// show")
+ def isShowRaw = code.lines exists (_.trim endsWith "// raw")
+
+ // old style
+ beSilentDuring(parse(code)) foreach { ts =>
+ ts foreach { t =>
+ withoutUnwrapping(logDebug(asCompactString(t)))
+ }
}
}
+
// debugging
def debugging[T](msg: String)(res: T) = {
- DBG(msg + " " + res)
+ logDebug(msg + " " + res)
res
}
- def DBG(s: => String) = if (isReplDebug) {
- //try repldbg(s)
- //catch { case x: AssertionError => repldbg("Assertion error printing debug string:\n " + x) }
- }
}
/** Utility methods for the Interpreter. */
object SparkIMain {
// The two name forms this is catching are the two sides of this assignment:
//
- // $line3.$read.$iw.$iw.Bippy =
+ // $line3.$read.$iw.$iw.Bippy =
// $line3.$read$$iw$$iw$Bippy@4a6a00ca
private def removeLineWrapper(s: String) = s.replaceAll("""\$line\d+[./]\$(read|eval|print)[$.]""", "")
private def removeIWPackages(s: String) = s.replaceAll("""\$(iw|iwC|read|eval|print)[$.]""", "")
private def removeSparkVals(s: String) = s.replaceAll("""\$VAL[0-9]+[$.]""", "")
+
def stripString(s: String) = removeSparkVals(removeIWPackages(removeLineWrapper(s)))
-
+
trait CodeAssembler[T] {
def preamble: String
def generate: T => String
@@ -1109,7 +1260,7 @@ object SparkIMain {
code println postamble
}
}
-
+
trait StrippingWriter {
def isStripping: Boolean
def stripImpl(str: String): String
@@ -1119,17 +1270,17 @@ object SparkIMain {
def maxStringLength: Int
def isTruncating: Boolean
def truncate(str: String): String = {
- if (isTruncating && str.length > maxStringLength)
+ if (isTruncating && (maxStringLength != 0 && str.length > maxStringLength))
(str take maxStringLength - 3) + "..."
else str
}
}
- abstract class StrippingTruncatingWriter(out: PrintWriter)
- extends PrintWriter(out)
+ abstract class StrippingTruncatingWriter(out: JPrintWriter)
+ extends JPrintWriter(out)
with StrippingWriter
with TruncatingWriter {
self =>
-
+
def clean(str: String): String = truncate(strip(str))
override def write(str: String) = super.write(clean(str))
}
@@ -1139,18 +1290,7 @@ object SparkIMain {
def isStripping = isettings.unwrapStrings
def isTruncating = reporter.truncationOK
- def stripImpl(str: String): String = {
- val cleaned = stripString(str)
- var ctrlChars = 0
- cleaned map { ch =>
- if (ch.isControl && !ch.isWhitespace) {
- ctrlChars += 1
- if (ctrlChars > 5) return "[line elided for control chars: possibly a scala signature]"
- else '?'
- }
- else ch
- }
- }
+ def stripImpl(str: String): String = naming.unmangle(str)
}
class ReplReporter(intp: SparkIMain) extends ConsoleReporter(intp.settings, null, new ReplStrippingWriter(intp)) {
@@ -1163,5 +1303,55 @@ object SparkIMain {
}
else Console.println(msg)
}
- }
+ }
+}
+
+class SparkISettings(intp: SparkIMain) extends Logging {
+ /** A list of paths where :load should look */
+ var loadPath = List(".")
+
+ /** Set this to true to see repl machinery under -Yrich-exceptions.
+ */
+ var showInternalStackTraces = false
+
+ /** The maximum length of toString to use when printing the result
+ * of an evaluation. 0 means no maximum. If a printout requires
+ * more than this number of characters, then the printout is
+ * truncated.
+ */
+ var maxPrintString = 800
+
+ /** The maximum number of completion candidates to print for tab
+ * completion without requiring confirmation.
+ */
+ var maxAutoprintCompletion = 250
+
+ /** String unwrapping can be disabled if it is causing issues.
+ * Settings this to false means you will see Strings like "$iw.$iw.".
+ */
+ var unwrapStrings = true
+
+ def deprecation_=(x: Boolean) = {
+ val old = intp.settings.deprecation.value
+ intp.settings.deprecation.value = x
+ if (!old && x) logDebug("Enabled -deprecation output.")
+ else if (old && !x) logDebug("Disabled -deprecation output.")
+ }
+
+ def deprecation: Boolean = intp.settings.deprecation.value
+
+ def allSettings = Map(
+ "maxPrintString" -> maxPrintString,
+ "maxAutoprintCompletion" -> maxAutoprintCompletion,
+ "unwrapStrings" -> unwrapStrings,
+ "deprecation" -> deprecation
+ )
+
+ private def allSettingsString =
+ allSettings.toList sortBy (_._1) map { case (k, v) => " " + k + " = " + v + "\n" } mkString
+
+ override def toString = """
+ | SparkISettings {
+ | %s
+ | }""".stripMargin.format(allSettingsString)
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala
deleted file mode 100644
index 605b7b259b..0000000000
--- a/repl/src/main/scala/org/apache/spark/repl/SparkISettings.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
- * @author Alexander Spoon
- */
-
-package org.apache.spark.repl
-
-import scala.tools.nsc._
-import scala.tools.nsc.interpreter._
-
-/** Settings for the interpreter
- *
- * @version 1.0
- * @author Lex Spoon, 2007/3/24
- **/
-class SparkISettings(intp: SparkIMain) {
- /** A list of paths where :load should look */
- var loadPath = List(".")
-
- /** Set this to true to see repl machinery under -Yrich-exceptions.
- */
- var showInternalStackTraces = false
-
- /** The maximum length of toString to use when printing the result
- * of an evaluation. 0 means no maximum. If a printout requires
- * more than this number of characters, then the printout is
- * truncated.
- */
- var maxPrintString = 800
-
- /** The maximum number of completion candidates to print for tab
- * completion without requiring confirmation.
- */
- var maxAutoprintCompletion = 250
-
- /** String unwrapping can be disabled if it is causing issues.
- * Settings this to false means you will see Strings like "$iw.$iw.".
- */
- var unwrapStrings = true
-
- def deprecation_=(x: Boolean) = {
- val old = intp.settings.deprecation.value
- intp.settings.deprecation.value = x
- if (!old && x) println("Enabled -deprecation output.")
- else if (old && !x) println("Disabled -deprecation output.")
- }
- def deprecation: Boolean = intp.settings.deprecation.value
-
- def allSettings = Map(
- "maxPrintString" -> maxPrintString,
- "maxAutoprintCompletion" -> maxAutoprintCompletion,
- "unwrapStrings" -> unwrapStrings,
- "deprecation" -> deprecation
- )
-
- private def allSettingsString =
- allSettings.toList sortBy (_._1) map { case (k, v) => " " + k + " = " + v + "\n" } mkString
-
- override def toString = """
- | SparkISettings {
- | %s
- | }""".stripMargin.format(allSettingsString)
-}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
index 41a1731d60..64084209e8 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Paul Phillips
*/
@@ -12,30 +12,34 @@ import scala.collection.{ mutable, immutable }
trait SparkImports {
self: SparkIMain =>
-
+
import global._
import definitions.{ ScalaPackage, JavaLangPackage, PredefModule }
import memberHandlers._
-
+
+ def isNoImports = settings.noimports.value
+ def isNoPredef = settings.nopredef.value
+
/** Synthetic import handlers for the language defined imports. */
private def makeWildcardImportHandler(sym: Symbol): ImportHandler = {
val hd :: tl = sym.fullName.split('.').toList map newTermName
val tree = Import(
tl.foldLeft(Ident(hd): Tree)((x, y) => Select(x, y)),
- List(ImportSelector(nme.WILDCARD, -1, null, -1))
+ ImportSelector.wildList
)
tree setSymbol sym
new ImportHandler(tree)
}
-
+
/** Symbols whose contents are language-defined to be imported. */
def languageWildcardSyms: List[Symbol] = List(JavaLangPackage, ScalaPackage, PredefModule)
def languageWildcards: List[Type] = languageWildcardSyms map (_.tpe)
def languageWildcardHandlers = languageWildcardSyms map makeWildcardImportHandler
-
- def importedTerms = onlyTerms(importHandlers flatMap (_.importedNames))
- def importedTypes = onlyTypes(importHandlers flatMap (_.importedNames))
-
+
+ def allImportedNames = importHandlers flatMap (_.importedNames)
+ def importedTerms = onlyTerms(allImportedNames)
+ def importedTypes = onlyTypes(allImportedNames)
+
/** Types which have been wildcard imported, such as:
* val x = "abc" ; import x._ // type java.lang.String
* import java.lang.String._ // object java.lang.String
@@ -48,30 +52,28 @@ trait SparkImports {
* into the compiler scopes.
*/
def sessionWildcards: List[Type] = {
- importHandlers flatMap {
- case x if x.importsWildcard => x.targetType
- case _ => None
- } distinct
+ importHandlers filter (_.importsWildcard) map (_.targetType) distinct
}
def wildcardTypes = languageWildcards ++ sessionWildcards
-
+
def languageSymbols = languageWildcardSyms flatMap membersAtPickler
def sessionImportedSymbols = importHandlers flatMap (_.importedSymbols)
def importedSymbols = languageSymbols ++ sessionImportedSymbols
def importedTermSymbols = importedSymbols collect { case x: TermSymbol => x }
def importedTypeSymbols = importedSymbols collect { case x: TypeSymbol => x }
def implicitSymbols = importedSymbols filter (_.isImplicit)
-
- def importedTermNamed(name: String) = importedTermSymbols find (_.name.toString == name)
+
+ def importedTermNamed(name: String): Symbol =
+ importedTermSymbols find (_.name.toString == name) getOrElse NoSymbol
/** Tuples of (source, imported symbols) in the order they were imported.
*/
def importedSymbolsBySource: List[(Symbol, List[Symbol])] = {
val lang = languageWildcardSyms map (sym => (sym, membersAtPickler(sym)))
- val session = importHandlers filter (_.targetType.isDefined) map { mh =>
- (mh.targetType.get.typeSymbol, mh.importedSymbols)
+ val session = importHandlers filter (_.targetType != NoType) map { mh =>
+ (mh.targetType.typeSymbol, mh.importedSymbols)
}
-
+
lang ++ session
}
def implicitSymbolsBySource: List[(Symbol, List[Symbol])] = {
@@ -79,7 +81,7 @@ trait SparkImports {
case (k, vs) => (k, vs filter (_.isImplicit))
} filterNot (_._2.isEmpty)
}
-
+
/** Compute imports that allow definitions from previous
* requests to be visible in a new request. Returns
* three pieces of related code:
@@ -90,7 +92,7 @@ trait SparkImports {
* 2. A code fragment that should go after the code
* of the new request.
*
- * 3. An access path which can be traverested to access
+ * 3. An access path which can be traversed to access
* any bindings inside code wrapped by #1 and #2 .
*
* The argument is a set of Names that need to be imported.
@@ -103,27 +105,27 @@ trait SparkImports {
* (3) It imports multiple same-named implicits, but only the
* last one imported is actually usable.
*/
- case class ComputedImports(prepend: String, append: String, access: String)
- protected def importsCode(wanted: Set[Name]): ComputedImports = {
- /** Narrow down the list of requests from which imports
+ case class SparkComputedImports(prepend: String, append: String, access: String)
+
+ protected def importsCode(wanted: Set[Name]): SparkComputedImports = {
+ /** Narrow down the list of requests from which imports
* should be taken. Removes requests which cannot contribute
* useful imports for the specified set of wanted names.
*/
case class ReqAndHandler(req: Request, handler: MemberHandler) { }
-
- def reqsToUse: List[ReqAndHandler] = {
+
+ def reqsToUse: List[ReqAndHandler] = {
/** Loop through a list of MemberHandlers and select which ones to keep.
* 'wanted' is the set of names that need to be imported.
*/
def select(reqs: List[ReqAndHandler], wanted: Set[Name]): List[ReqAndHandler] = {
- val isWanted = wanted contains _
// Single symbol imports might be implicits! See bug #1752. Rather than
// try to finesse this, we will mimic all imports for now.
def keepHandler(handler: MemberHandler) = handler match {
case _: ImportHandler => true
- case x => x.definesImplicit || (x.definedNames exists isWanted)
+ case x => x.definesImplicit || (x.definedNames exists wanted)
}
-
+
reqs match {
case Nil => Nil
case rh :: rest if !keepHandler(rh.handler) => select(rest, wanted)
@@ -133,7 +135,7 @@ trait SparkImports {
rh :: select(rest, newWanted)
}
}
-
+
/** Flatten the handlers out and pair each with the original request */
select(allReqAndHandlers reverseMap { case (r, h) => ReqAndHandler(r, h) }, wanted).reverse
}
@@ -147,8 +149,13 @@ trait SparkImports {
code append "class %sC extends Serializable {\n".format(impname)
trailingBraces append "}\nval " + impname + " = new " + impname + "C;\n"
accessPath append ("." + impname)
-
+
currentImps.clear
+ // code append "object %s {\n".format(impname)
+ // trailingBraces append "}\n"
+ // accessPath append ("." + impname)
+
+ // currentImps.clear
}
addWrapper()
@@ -159,36 +166,33 @@ trait SparkImports {
// If the user entered an import, then just use it; add an import wrapping
// level if the import might conflict with some other import
case x: ImportHandler =>
- if (x.importsWildcard || (currentImps exists (x.importedNames contains _)))
+ if (x.importsWildcard || currentImps.exists(x.importedNames contains _))
addWrapper()
-
+
code append (x.member + "\n")
-
+
// give wildcard imports a import wrapper all to their own
if (x.importsWildcard) addWrapper()
else currentImps ++= x.importedNames
// For other requests, import each defined name.
// import them explicitly instead of with _, so that
- // ambiguity errors will not be generated. Also, quote
- // the name of the variable, so that we don't need to
- // handle quoting keywords separately.
+ // ambiguity errors will not be generated. Also, quote
+ // the name of the variable, so that we don't need to
+ // handle quoting keywords separately.
case x =>
for (imv <- x.definedNames) {
- // MATEI: Changed this check because it was messing up for case classes
- // (trying to import them twice within the same wrapper), but that is more likely
- // due to a miscomputation of names that makes the code think they're unique.
- // Need to evaluate whether having so many wrappers is a bad thing.
- /*if (currentImps contains imv)*/
- val imvName = imv.toString
- if (currentImps exists (_.toString == imvName)) addWrapper()
-
+ if (currentImps contains imv) addWrapper()
val objName = req.lineRep.readPath
val valName = "$VAL" + newValId();
- code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
- code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
-
- //code append ("import %s\n" format (req fullPath imv))
+
+ if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported
+ code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
+ code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
+ }
+ // code.append("val " + valName + " = " + objName + ".INSTANCE;\n")
+ // code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n")
+ // code append ("import " + (req fullPath imv) + "\n")
currentImps += imv
}
}
@@ -196,14 +200,14 @@ trait SparkImports {
// add one extra wrapper, to prevent warnings in the common case of
// redefining the value bound in the last interpreter request.
addWrapper()
- ComputedImports(code.toString, trailingBraces.toString, accessPath.toString)
+ SparkComputedImports(code.toString, trailingBraces.toString, accessPath.toString)
}
-
+
private def allReqAndHandlers =
prevRequestList flatMap (req => req.handlers map (req -> _))
private def membersAtPickler(sym: Symbol): List[Symbol] =
- atPickler(sym.info.nonPrivateMembers)
+ beforePickler(sym.info.nonPrivateMembers.toList)
private var curValId = 0
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
index fdc172d753..8865f82bc0 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Paul Phillips
*/
@@ -11,30 +11,31 @@ import scala.tools.nsc.interpreter._
import scala.tools.jline._
import scala.tools.jline.console.completer._
import Completion._
-import collection.mutable.ListBuffer
+import scala.collection.mutable.ListBuffer
+import org.apache.spark.Logging
// REPL completor - queries supplied interpreter for valid
// completions based on current contents of buffer.
-class SparkJLineCompletion(val intp: SparkIMain) extends Completion with CompletionOutput {
+class SparkJLineCompletion(val intp: SparkIMain) extends Completion with CompletionOutput with Logging {
val global: intp.global.type = intp.global
import global._
- import definitions.{ PredefModule, RootClass, AnyClass, AnyRefClass, ScalaPackage, JavaLangPackage }
+ import definitions.{ PredefModule, AnyClass, AnyRefClass, ScalaPackage, JavaLangPackage }
+ import rootMirror.{ RootClass, getModuleIfDefined }
type ExecResult = Any
- import intp.{ DBG, debugging, afterTyper }
-
+ import intp.{ debugging }
+
// verbosity goes up with consecutive tabs
private var verbosity: Int = 0
def resetVerbosity() = verbosity = 0
-
- def getType(name: String, isModule: Boolean) = {
- val f = if (isModule) definitions.getModule(_: Name) else definitions.getClass(_: Name)
- try Some(f(name).tpe)
- catch { case _: MissingRequirementError => None }
- }
-
- def typeOf(name: String) = getType(name, false)
- def moduleOf(name: String) = getType(name, true)
-
+
+ def getSymbol(name: String, isModule: Boolean) = (
+ if (isModule) getModuleIfDefined(name)
+ else getModuleIfDefined(name)
+ )
+ def getType(name: String, isModule: Boolean) = getSymbol(name, isModule).tpe
+ def typeOf(name: String) = getType(name, false)
+ def moduleOf(name: String) = getType(name, true)
+
trait CompilerCompletion {
def tp: Type
def effectiveTp = tp match {
@@ -48,16 +49,16 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
private def anyMembers = AnyClass.tpe.nonPrivateMembers
def anyRefMethodsToShow = Set("isInstanceOf", "asInstanceOf", "toString")
- def tos(sym: Symbol) = sym.name.decode.toString
- def memberNamed(s: String) = members find (x => tos(x) == s)
- def hasMethod(s: String) = methods exists (x => tos(x) == s)
+ def tos(sym: Symbol): String = sym.decodedName
+ def memberNamed(s: String) = afterTyper(effectiveTp member newTermName(s))
+ def hasMethod(s: String) = memberNamed(s).isMethod
// XXX we'd like to say "filterNot (_.isDeprecated)" but this causes the
// compiler to crash for reasons not yet known.
- def members = afterTyper((effectiveTp.nonPrivateMembers ++ anyMembers) filter (_.isPublic))
- def methods = members filter (_.isMethod)
- def packages = members filter (_.isPackage)
- def aliases = members filter (_.isAliasType)
+ def members = afterTyper((effectiveTp.nonPrivateMembers.toList ++ anyMembers) filter (_.isPublic))
+ def methods = members.toList filter (_.isMethod)
+ def packages = members.toList filter (_.isPackage)
+ def aliases = members.toList filter (_.isAliasType)
def memberNames = members map tos
def methodNames = methods map tos
@@ -65,12 +66,19 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
def aliasNames = aliases map tos
}
+ object NoTypeCompletion extends TypeMemberCompletion(NoType) {
+ override def memberNamed(s: String) = NoSymbol
+ override def members = Nil
+ override def follow(s: String) = None
+ override def alternativesFor(id: String) = Nil
+ }
+
object TypeMemberCompletion {
def apply(tp: Type, runtimeType: Type, param: NamedParam): TypeMemberCompletion = {
new TypeMemberCompletion(tp) {
var upgraded = false
lazy val upgrade = {
- intp rebind param
+ intp rebind param
intp.reporter.printMessage("\nRebinding stable value %s from %s to %s".format(param.name, tp, param.tpe))
upgraded = true
new TypeMemberCompletion(runtimeType)
@@ -92,7 +100,8 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
}
}
def apply(tp: Type): TypeMemberCompletion = {
- if (tp.typeSymbol.isPackageClass) new PackageCompletion(tp)
+ if (tp eq NoType) NoTypeCompletion
+ else if (tp.typeSymbol.isPackageClass) new PackageCompletion(tp)
else new TypeMemberCompletion(tp)
}
def imported(tp: Type) = new ImportCompletion(tp)
@@ -103,9 +112,9 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
def excludeEndsWith: List[String] = Nil
def excludeStartsWith: List[String] = List("<") // <byname>, <repeated>, etc.
def excludeNames: List[String] = (anyref.methodNames filterNot anyRefMethodsToShow) :+ "_root_"
-
+
def methodSignatureString(sym: Symbol) = {
- SparkIMain stripString afterTyper(new MethodSymbolOutput(sym).methodString())
+ IMain stripString afterTyper(new MethodSymbolOutput(sym).methodString())
}
def exclude(name: String): Boolean = (
@@ -118,10 +127,10 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
def completions(verbosity: Int) =
debugging(tp + " completions ==> ")(filtered(memberNames))
-
+
override def follow(s: String): Option[CompletionAware] =
- debugging(tp + " -> '" + s + "' ==> ")(memberNamed(s) map (x => TypeMemberCompletion(x.tpe)))
-
+ debugging(tp + " -> '" + s + "' ==> ")(Some(TypeMemberCompletion(memberNamed(s).tpe)) filterNot (_ eq NoTypeCompletion))
+
override def alternativesFor(id: String): List[String] =
debugging(id + " alternatives ==> ") {
val alts = members filter (x => x.isMethod && tos(x) == id) map methodSignatureString
@@ -131,7 +140,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
override def toString = "%s (%d members)".format(tp, members.size)
}
-
+
class PackageCompletion(tp: Type) extends TypeMemberCompletion(tp) {
override def excludeNames = anyref.methodNames
}
@@ -142,43 +151,44 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
case _ => memberNames
}
}
-
+
class ImportCompletion(tp: Type) extends TypeMemberCompletion(tp) {
override def completions(verbosity: Int) = verbosity match {
case 0 => filtered(members filterNot (_.isSetter) map tos)
case _ => super.completions(verbosity)
}
}
-
+
// not for completion but for excluding
object anyref extends TypeMemberCompletion(AnyRefClass.tpe) { }
-
+
// the unqualified vals/defs/etc visible in the repl
object ids extends CompletionAware {
override def completions(verbosity: Int) = intp.unqualifiedIds ++ List("classOf") //, "_root_")
// now we use the compiler for everything.
- override def follow(id: String) = {
- if (completions(0) contains id) {
- intp typeOfExpression id map { tpe =>
- def default = TypeMemberCompletion(tpe)
-
- // only rebinding vals in power mode for now.
- if (!isReplPower) default
- else intp runtimeClassAndTypeOfTerm id match {
- case Some((clazz, runtimeType)) =>
- val sym = intp.symbolOfTerm(id)
- if (sym.isStable) {
- val param = new NamedParam.Untyped(id, intp valueOfTerm id getOrElse null)
- TypeMemberCompletion(tpe, runtimeType, param)
- }
- else default
- case _ =>
- default
+ override def follow(id: String): Option[CompletionAware] = {
+ if (!completions(0).contains(id))
+ return None
+
+ val tpe = intp typeOfExpression id
+ if (tpe == NoType)
+ return None
+
+ def default = Some(TypeMemberCompletion(tpe))
+
+ // only rebinding vals in power mode for now.
+ if (!isReplPower) default
+ else intp runtimeClassAndTypeOfTerm id match {
+ case Some((clazz, runtimeType)) =>
+ val sym = intp.symbolOfTerm(id)
+ if (sym.isStable) {
+ val param = new NamedParam.Untyped(id, intp valueOfTerm id getOrElse null)
+ Some(TypeMemberCompletion(tpe, runtimeType, param))
}
- }
+ else default
+ case _ =>
+ default
}
- else
- None
}
override def toString = "<repl ids> (%s)".format(completions(0).size)
}
@@ -187,17 +197,10 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
private def imported = intp.sessionWildcards map TypeMemberCompletion.imported
// literal Ints, Strings, etc.
- object literals extends CompletionAware {
- def simpleParse(code: String): Tree = {
- val unit = new CompilationUnit(new util.BatchSourceFile("<console>", code))
- val scanner = new syntaxAnalyzer.UnitParser(unit)
- val tss = scanner.templateStatSeq(false)._2
-
- if (tss.size == 1) tss.head else EmptyTree
- }
-
+ object literals extends CompletionAware {
+ def simpleParse(code: String): Tree = newUnitParser(code).templateStats().last
def completions(verbosity: Int) = Nil
-
+
override def follow(id: String) = simpleParse(id) match {
case x: Literal => Some(new LiteralCompletion(x))
case _ => None
@@ -210,18 +213,18 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
override def follow(id: String) = id match {
case "_root_" => Some(this)
case _ => super.follow(id)
- }
+ }
}
// members of Predef
object predef extends TypeMemberCompletion(PredefModule.tpe) {
override def excludeEndsWith = super.excludeEndsWith ++ List("Wrapper", "ArrayOps")
override def excludeStartsWith = super.excludeStartsWith ++ List("wrap")
override def excludeNames = anyref.methodNames
-
+
override def exclude(name: String) = super.exclude(name) || (
(name contains "2")
)
-
+
override def completions(verbosity: Int) = verbosity match {
case 0 => Nil
case _ => super.completions(verbosity)
@@ -234,7 +237,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
override def exclude(name: String) = super.exclude(name) || (
skipArity(name)
)
-
+
override def completions(verbosity: Int) = verbosity match {
case 0 => filtered(packageNames ++ aliasNames)
case _ => super.completions(verbosity)
@@ -244,7 +247,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
object javalang extends PackageCompletion(JavaLangPackage.tpe) {
override lazy val excludeEndsWith = super.excludeEndsWith ++ List("Exception", "Error")
override lazy val excludeStartsWith = super.excludeStartsWith ++ List("CharacterData")
-
+
override def completions(verbosity: Int) = verbosity match {
case 0 => filtered(packageNames)
case _ => super.completions(verbosity)
@@ -256,7 +259,7 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
lazy val topLevelBase: List[CompletionAware] = List(ids, rootClass, predef, scalalang, javalang, literals)
def topLevel = topLevelBase ++ imported
def topLevelThreshold = 50
-
+
// the first tier of top level objects (doesn't include file completion)
def topLevelFor(parsed: Parsed): List[String] = {
val buf = new ListBuffer[String]
@@ -280,19 +283,6 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
if (parsed.isEmpty) xs map ("." + _) else xs
}
- // chasing down results which won't parse
- def execute(line: String): Option[ExecResult] = {
- val parsed = Parsed(line)
- def noDotOrSlash = line forall (ch => ch != '.' && ch != '/')
-
- if (noDotOrSlash) None // we defer all unqualified ids to the repl.
- else {
- (ids executionFor parsed) orElse
- (rootClass executionFor parsed) orElse
- (FileCompletion executionFor line)
- }
- }
-
// generic interface for querying (e.g. interpreter loop, testing)
def completions(buf: String): List[String] =
topLevelFor(Parsed.dotted(buf + ".", buf.length + 1))
@@ -327,11 +317,11 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
// This is jline's entry point for completion.
override def complete(buf: String, cursor: Int): Candidates = {
verbosity = if (isConsecutiveTabs(buf, cursor)) verbosity + 1 else 0
- DBG("\ncomplete(%s, %d) last = (%s, %d), verbosity: %s".format(buf, cursor, lastBuf, lastCursor, verbosity))
+ logDebug("\ncomplete(%s, %d) last = (%s, %d), verbosity: %s".format(buf, cursor, lastBuf, lastCursor, verbosity))
// we don't try lower priority completions unless higher ones return no results.
def tryCompletion(p: Parsed, completionFunction: Parsed => List[String]): Option[Candidates] = {
- val winners = completionFunction(p)
+ val winners = completionFunction(p)
if (winners.isEmpty)
return None
val newCursor =
@@ -340,39 +330,45 @@ class SparkJLineCompletion(val intp: SparkIMain) extends Completion with Complet
val advance = commonPrefix(winners)
lastCursor = p.position + advance.length
lastBuf = (buf take p.position) + advance
- DBG("tryCompletion(%s, _) lastBuf = %s, lastCursor = %s, p.position = %s".format(
+ logDebug("tryCompletion(%s, _) lastBuf = %s, lastCursor = %s, p.position = %s".format(
p, lastBuf, lastCursor, p.position))
p.position
}
-
+
Some(Candidates(newCursor, winners))
}
-
+
def mkDotted = Parsed.dotted(buf, cursor) withVerbosity verbosity
def mkUndelimited = Parsed.undelimited(buf, cursor) withVerbosity verbosity
// a single dot is special cased to completion on the previous result
def lastResultCompletion =
- if (!looksLikeInvocation(buf)) None
+ if (!looksLikeInvocation(buf)) None
else tryCompletion(Parsed.dotted(buf drop 1, cursor), lastResultFor)
- def regularCompletion = tryCompletion(mkDotted, topLevelFor)
- def fileCompletion =
- if (!looksLikePath(buf)) None
- else tryCompletion(mkUndelimited, FileCompletion completionsFor _.buffer)
-
- /** This is the kickoff point for all manner of theoretically possible compiler
- * unhappiness - fault may be here or elsewhere, but we don't want to crash the
- * repl regardless. Hopefully catching Exception is enough, but because the
- * compiler still throws some Errors it may not be.
+ def tryAll = (
+ lastResultCompletion
+ orElse tryCompletion(mkDotted, topLevelFor)
+ getOrElse Candidates(cursor, Nil)
+ )
+
+ /**
+ * This is the kickoff point for all manner of theoretically
+ * possible compiler unhappiness. The fault may be here or
+ * elsewhere, but we don't want to crash the repl regardless.
+ * The compiler makes it impossible to avoid catching Throwable
+ * with its unfortunate tendency to throw java.lang.Errors and
+ * AssertionErrors as the hats drop. We take two swings at it
+ * because there are some spots which like to throw an assertion
+ * once, then work after that. Yeah, what can I say.
*/
- try {
- (lastResultCompletion orElse regularCompletion orElse fileCompletion) getOrElse Candidates(cursor, Nil)
- }
- catch {
- case ex: Exception =>
- DBG("Error: complete(%s, %s) provoked %s".format(buf, cursor, ex))
- Candidates(cursor, List(" ", "<completion error: " + ex.getMessage + ">"))
+ try tryAll
+ catch { case ex: Throwable =>
+ logWarning("Error: complete(%s, %s) provoked".format(buf, cursor) + ex)
+ Candidates(cursor,
+ if (isReplDebug) List("<error:" + ex + ">")
+ else Nil
+ )
}
}
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
index d9e1de105c..60a4d7841e 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Stepan Koltsov
*/
@@ -15,28 +15,33 @@ import scala.collection.JavaConverters._
import Completion._
import io.Streamable.slurp
-/** Reads from the console using JLine */
-class SparkJLineReader(val completion: Completion) extends InteractiveReader {
+/**
+ * Reads from the console using JLine.
+ */
+class SparkJLineReader(_completion: => Completion) extends InteractiveReader {
val interactive = true
+ val consoleReader = new JLineConsoleReader()
+
+ lazy val completion = _completion
lazy val history: JLineHistory = JLineHistory()
- lazy val keyBindings =
- try KeyBinding parse slurp(term.getDefaultBindings)
- catch { case _: Exception => Nil }
private def term = consoleReader.getTerminal()
def reset() = term.reset()
def init() = term.init()
-
+
def scalaToJline(tc: ScalaCompleter): Completer = new Completer {
def complete(_buf: String, cursor: Int, candidates: JList[CharSequence]): Int = {
- val buf = if (_buf == null) "" else _buf
+ val buf = if (_buf == null) "" else _buf
val Candidates(newCursor, newCandidates) = tc.complete(buf, cursor)
newCandidates foreach (candidates add _)
newCursor
}
}
-
+
class JLineConsoleReader extends ConsoleReader with ConsoleReaderHelper {
+ if ((history: History) ne NoHistory)
+ this setHistory history
+
// working around protected/trait/java insufficiencies.
def goBack(num: Int): Unit = back(num)
def readOneKey(prompt: String) = {
@@ -46,34 +51,28 @@ class SparkJLineReader(val completion: Completion) extends InteractiveReader {
}
def eraseLine() = consoleReader.resetPromptLine("", "", 0)
def redrawLineAndFlush(): Unit = { flush() ; drawLine() ; flush() }
-
- this setBellEnabled false
- if (history ne NoHistory)
- this setHistory history
-
- if (completion ne NoCompletion) {
- val argCompletor: ArgumentCompleter =
- new ArgumentCompleter(new JLineDelimiter, scalaToJline(completion.completer()))
- argCompletor setStrict false
-
- this addCompleter argCompletor
- this setAutoprintThreshold 400 // max completion candidates without warning
+ // override def readLine(prompt: String): String
+
+ // A hook for running code after the repl is done initializing.
+ lazy val postInit: Unit = {
+ this setBellEnabled false
+
+ if (completion ne NoCompletion) {
+ val argCompletor: ArgumentCompleter =
+ new ArgumentCompleter(new JLineDelimiter, scalaToJline(completion.completer()))
+ argCompletor setStrict false
+
+ this addCompleter argCompletor
+ this setAutoprintThreshold 400 // max completion candidates without warning
+ }
}
}
-
- val consoleReader: JLineConsoleReader = new JLineConsoleReader()
- def currentLine: String = consoleReader.getCursorBuffer.buffer.toString
+ def currentLine = consoleReader.getCursorBuffer.buffer.toString
def redrawLine() = consoleReader.redrawLineAndFlush()
- def eraseLine() = {
- while (consoleReader.delete()) { }
- // consoleReader.eraseLine()
- }
+ def eraseLine() = consoleReader.eraseLine()
+ // Alternate implementation, not sure if/when I need this.
+ // def eraseLine() = while (consoleReader.delete()) { }
def readOneLine(prompt: String) = consoleReader readLine prompt
def readOneKey(prompt: String) = consoleReader readOneKey prompt
}
-
-object SparkJLineReader {
- def apply(intp: SparkIMain): SparkJLineReader = apply(new SparkJLineCompletion(intp))
- def apply(comp: Completion): SparkJLineReader = new SparkJLineReader(comp)
-}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
index a3409bf665..382f8360a7 100644
--- a/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala
@@ -1,5 +1,5 @@
/* NSC -- new Scala compiler
- * Copyright 2005-2011 LAMP/EPFL
+ * Copyright 2005-2013 LAMP/EPFL
* @author Martin Odersky
*/
@@ -10,13 +10,14 @@ import scala.tools.nsc.interpreter._
import scala.collection.{ mutable, immutable }
import scala.PartialFunction.cond
-import scala.reflect.NameTransformer
-import util.Chars
+import scala.reflect.internal.Chars
+import scala.reflect.internal.Flags._
+import scala.language.implicitConversions
trait SparkMemberHandlers {
val intp: SparkIMain
- import intp.{ Request, global, naming, atPickler }
+ import intp.{ Request, global, naming }
import global._
import naming._
@@ -29,7 +30,7 @@ trait SparkMemberHandlers {
front + (xs map string2codeQuoted mkString " + ")
}
private implicit def name2string(name: Name) = name.toString
-
+
/** A traverser that finds all mentioned identifiers, i.e. things
* that need to be imported. It might return extra names.
*/
@@ -54,26 +55,28 @@ trait SparkMemberHandlers {
}
def chooseHandler(member: Tree): MemberHandler = member match {
- case member: DefDef => new DefHandler(member)
- case member: ValDef => new ValHandler(member)
- case member@Assign(Ident(_), _) => new AssignHandler(member)
- case member: ModuleDef => new ModuleHandler(member)
- case member: ClassDef => new ClassHandler(member)
- case member: TypeDef => new TypeAliasHandler(member)
- case member: Import => new ImportHandler(member)
- case DocDef(_, documented) => chooseHandler(documented)
- case member => new GenericHandler(member)
+ case member: DefDef => new DefHandler(member)
+ case member: ValDef => new ValHandler(member)
+ case member: Assign => new AssignHandler(member)
+ case member: ModuleDef => new ModuleHandler(member)
+ case member: ClassDef => new ClassHandler(member)
+ case member: TypeDef => new TypeAliasHandler(member)
+ case member: Import => new ImportHandler(member)
+ case DocDef(_, documented) => chooseHandler(documented)
+ case member => new GenericHandler(member)
}
-
+
sealed abstract class MemberDefHandler(override val member: MemberDef) extends MemberHandler(member) {
+ def symbol = if (member.symbol eq null) NoSymbol else member.symbol
def name: Name = member.name
def mods: Modifiers = member.mods
def keyword = member.keyword
- def prettyName = NameTransformer.decode(name)
-
+ def prettyName = name.decode
+
override def definesImplicit = member.mods.isImplicit
override def definesTerm: Option[TermName] = Some(name.toTermName) filter (_ => name.isTermName)
override def definesType: Option[TypeName] = Some(name.toTypeName) filter (_ => name.isTypeName)
+ override def definedSymbols = if (symbol eq NoSymbol) Nil else List(symbol)
}
/** Class to handle one member among all the members included
@@ -82,11 +85,8 @@ trait SparkMemberHandlers {
sealed abstract class MemberHandler(val member: Tree) {
def definesImplicit = false
def definesValue = false
- def isLegalTopLevel = member match {
- case _: ModuleDef | _: ClassDef | _: Import => true
- case _ => false
- }
-
+ def isLegalTopLevel = false
+
def definesTerm = Option.empty[TermName]
def definesType = Option.empty[TypeName]
@@ -94,6 +94,7 @@ trait SparkMemberHandlers {
def importedNames = List[Name]()
def definedNames = definesTerm.toList ++ definesType.toList
def definedOrImported = definedNames ++ importedNames
+ def definedSymbols = List[Symbol]()
def extraCodeToEvaluate(req: Request): String = ""
def resultExtractionCode(req: Request): String = ""
@@ -103,11 +104,11 @@ trait SparkMemberHandlers {
}
class GenericHandler(member: Tree) extends MemberHandler(member)
-
+
class ValHandler(member: ValDef) extends MemberDefHandler(member) {
- val maxStringElements = 1000 // no need to mkString billions of elements
+ val maxStringElements = 1000 // no need to mkString billions of elements
override def definesValue = true
-
+
override def resultExtractionCode(req: Request): String = {
val isInternal = isUserVarName(name) && req.lookupTypeOf(name) == "Unit"
if (!mods.isPublic || isInternal) ""
@@ -116,22 +117,27 @@ trait SparkMemberHandlers {
val resultString =
if (mods.isLazy) codegenln(false, "<lazy>")
else any2stringOf(req fullPath name, maxStringElements)
-
- """ + "%s: %s = " + %s""".format(prettyName, string2code(req typeOf name), resultString)
+
+ val vidString =
+ if (replProps.vids) """" + " @ " + "%%8x".format(System.identityHashCode(%s)) + " """.trim.format(req fullPath name)
+ else ""
+
+ """ + "%s%s: %s = " + %s""".format(string2code(prettyName), vidString, string2code(req typeOf name), resultString)
}
}
}
class DefHandler(member: DefDef) extends MemberDefHandler(member) {
private def vparamss = member.vparamss
- // true if 0-arity
- override def definesValue = vparamss.isEmpty || vparamss.head.isEmpty
+ private def isMacro = member.symbol hasFlag MACRO
+ // true if not a macro and 0-arity
+ override def definesValue = !isMacro && flattensToEmpty(vparamss)
override def resultExtractionCode(req: Request) =
if (mods.isPublic) codegenln(name, ": ", req.typeOf(name)) else ""
}
class AssignHandler(member: Assign) extends MemberHandler(member) {
- val lhs = member.lhs.asInstanceOf[Ident] // an unfortunate limitation
+ val Assign(lhs, rhs) = member
val name = newTermName(freshInternalVarName())
override def definesTerm = Some(name)
@@ -142,15 +148,15 @@ trait SparkMemberHandlers {
/** Print out lhs instead of the generated varName */
override def resultExtractionCode(req: Request) = {
val lhsType = string2code(req lookupTypeOf name)
- val res = string2code(req fullPath name)
-
- """ + "%s: %s = " + %s + "\n" """.format(lhs, lhsType, res) + "\n"
+ val res = string2code(req fullPath name)
+ """ + "%s: %s = " + %s + "\n" """.format(string2code(lhs.toString), lhsType, res) + "\n"
}
}
class ModuleHandler(module: ModuleDef) extends MemberDefHandler(module) {
override def definesTerm = Some(name)
override def definesValue = true
+ override def isLegalTopLevel = true
override def resultExtractionCode(req: Request) = codegenln("defined module ", name)
}
@@ -158,7 +164,8 @@ trait SparkMemberHandlers {
class ClassHandler(member: ClassDef) extends MemberDefHandler(member) {
override def definesType = Some(name.toTypeName)
override def definesTerm = Some(name.toTermName) filter (_ => mods.isCase)
-
+ override def isLegalTopLevel = true
+
override def resultExtractionCode(req: Request) =
codegenln("defined %s %s".format(keyword, name))
}
@@ -173,26 +180,42 @@ trait SparkMemberHandlers {
class ImportHandler(imp: Import) extends MemberHandler(imp) {
val Import(expr, selectors) = imp
- def targetType = intp.typeOfExpression("" + expr)
-
+ def targetType: Type = intp.typeOfExpression("" + expr)
+ override def isLegalTopLevel = true
+
+ def createImportForName(name: Name): String = {
+ selectors foreach {
+ case sel @ ImportSelector(old, _, `name`, _) => return "import %s.{ %s }".format(expr, sel)
+ case _ => ()
+ }
+ "import %s.%s".format(expr, name)
+ }
+ // TODO: Need to track these specially to honor Predef masking attempts,
+ // because they must be the leading imports in the code generated for each
+ // line. We can use the same machinery as Contexts now, anyway.
+ def isPredefImport = isReferenceToPredef(expr)
+
// wildcard imports, e.g. import foo._
private def selectorWild = selectors filter (_.name == nme.USCOREkw)
// renamed imports, e.g. import foo.{ bar => baz }
private def selectorRenames = selectors map (_.rename) filterNot (_ == null)
-
+
/** Whether this import includes a wildcard import */
val importsWildcard = selectorWild.nonEmpty
-
+
+ /** Whether anything imported is implicit .*/
+ def importsImplicit = implicitSymbols.nonEmpty
+
def implicitSymbols = importedSymbols filter (_.isImplicit)
def importedSymbols = individualSymbols ++ wildcardSymbols
-
+
lazy val individualSymbols: List[Symbol] =
- atPickler(targetType.toList flatMap (tp => individualNames map (tp nonPrivateMember _)))
+ beforePickler(individualNames map (targetType nonPrivateMember _))
lazy val wildcardSymbols: List[Symbol] =
- if (importsWildcard) atPickler(targetType.toList flatMap (_.nonPrivateMembers))
+ if (importsWildcard) beforePickler(targetType.nonPrivateMembers.toList)
else Nil
-
+
/** Complete list of names imported by a wildcard */
lazy val wildcardNames: List[Name] = wildcardSymbols map (_.name)
lazy val individualNames: List[Name] = selectorRenames filterNot (_ == nme.USCOREkw) flatMap (_.bothNames)
@@ -200,7 +223,7 @@ trait SparkMemberHandlers {
/** The names imported by this statement */
override lazy val importedNames: List[Name] = wildcardNames ++ individualNames
lazy val importsSymbolNamed: Set[String] = importedNames map (_.toString) toSet
-
+
def importString = imp.toString
override def resultExtractionCode(req: Request) = codegenln(importString) + "\n"
}
diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala
new file mode 100644
index 0000000000..7fd5fbb424
--- /dev/null
+++ b/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala
@@ -0,0 +1,32 @@
+/*
+ * 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.repl
+
+import scala.tools.nsc.Settings
+
+/**
+ * <i>scala.tools.nsc.Settings</i> implementation adding Spark-specific REPL
+ * command line options.
+ */
+class SparkRunnerSettings(error: String => Unit) extends Settings(error){
+
+ val loadfiles = MultiStringSetting(
+ "-i",
+ "file",
+ "load a file (assumes the code is given interactively)")
+}
diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index 6e4504d4d5..daaa2a0305 100644
--- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -1,20 +1,3 @@
-/*
- * 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.repl
import java.io._
@@ -58,12 +41,12 @@ class ReplSuite extends FunSuite {
def assertContains(message: String, output: String) {
assert(output.contains(message),
- "Interpreter output did not contain '" + message + "':\n" + output)
+ "Interpreter output did not contain '" + message + "':\n" + output)
}
def assertDoesNotContain(message: String, output: String) {
assert(!output.contains(message),
- "Interpreter output contained '" + message + "':\n" + output)
+ "Interpreter output contained '" + message + "':\n" + output)
}
test("propagation of local properties") {
@@ -95,96 +78,103 @@ class ReplSuite extends FunSuite {
System.clearProperty("spark.hostPort")
}
- test ("simple foreach with accumulator") {
- val output = runInterpreter("local", """
- val accum = sc.accumulator(0)
- sc.parallelize(1 to 10).foreach(x => accum += x)
- accum.value
- """)
+ test("simple foreach with accumulator") {
+ val output = runInterpreter("local",
+ """
+ |val accum = sc.accumulator(0)
+ |sc.parallelize(1 to 10).foreach(x => accum += x)
+ |accum.value
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res1: Int = 55", output)
}
- test ("external vars") {
- val output = runInterpreter("local", """
- var v = 7
- sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
- """)
+ test("external vars") {
+ val output = runInterpreter("local",
+ """
+ |var v = 7
+ |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => v).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
assertContains("res1: Int = 100", output)
}
- test ("external classes") {
- val output = runInterpreter("local", """
- class C {
- def foo = 5
- }
- sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_)
- """)
+ test("external classes") {
+ val output = runInterpreter("local",
+ """
+ |class C {
+ |def foo = 5
+ |}
+ |sc.parallelize(1 to 10).map(x => (new C).foo).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 50", output)
}
- test ("external functions") {
- val output = runInterpreter("local", """
- def double(x: Int) = x + x
- sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
- """)
+ test("external functions") {
+ val output = runInterpreter("local",
+ """
+ |def double(x: Int) = x + x
+ |sc.parallelize(1 to 10).map(x => double(x)).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 110", output)
}
- test ("external functions that access vars") {
- val output = runInterpreter("local", """
- var v = 7
- def getV() = v
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- """)
+ test("external functions that access vars") {
+ val output = runInterpreter("local",
+ """
+ |var v = 7
+ |def getV() = v
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
assertContains("res1: Int = 100", output)
}
- test ("broadcast vars") {
+ test("broadcast vars") {
// Test that the value that a broadcast var had when it was created is used,
// even if that variable is then modified in the driver program
// TODO: This doesn't actually work for arrays when we run in local mode!
- val output = runInterpreter("local", """
- var array = new Array[Int](5)
- val broadcastArray = sc.broadcast(array)
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- array(0) = 5
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- """)
+ val output = runInterpreter("local",
+ """
+ |var array = new Array[Int](5)
+ |val broadcastArray = sc.broadcast(array)
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ |array(0) = 5
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Array[Int] = Array(0, 0, 0, 0, 0)", output)
assertContains("res2: Array[Int] = Array(5, 0, 0, 0, 0)", output)
}
- test ("interacting with files") {
+ test("interacting with files") {
val tempDir = Files.createTempDir()
val out = new FileWriter(tempDir + "/input")
out.write("Hello world!\n")
out.write("What's up?\n")
out.write("Goodbye\n")
out.close()
- val output = runInterpreter("local", """
- var file = sc.textFile("%s/input").cache()
- file.count()
- file.count()
- file.count()
- """.format(tempDir.getAbsolutePath))
+ val output = runInterpreter("local",
+ """
+ |var file = sc.textFile("%s/input").cache()
+ |file.count()
+ |file.count()
+ |file.count()
+ """.stripMargin.format(tempDir.getAbsolutePath))
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Long = 3", output)
@@ -192,19 +182,20 @@ class ReplSuite extends FunSuite {
assertContains("res2: Long = 3", output)
}
- test ("local-cluster mode") {
- val output = runInterpreter("local-cluster[1,1,512]", """
- var v = 7
- def getV() = v
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- var array = new Array[Int](5)
- val broadcastArray = sc.broadcast(array)
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- array(0) = 5
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- """)
+ test("local-cluster mode") {
+ val output = runInterpreter("local-cluster[1,1,512]",
+ """
+ |var v = 7
+ |def getV() = v
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |var array = new Array[Int](5)
+ |val broadcastArray = sc.broadcast(array)
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ |array(0) = 5
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
@@ -214,19 +205,20 @@ class ReplSuite extends FunSuite {
}
if (System.getenv("MESOS_NATIVE_LIBRARY") != null) {
- test ("running on Mesos") {
- val output = runInterpreter("localquiet", """
- var v = 7
- def getV() = v
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- v = 10
- sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
- var array = new Array[Int](5)
- val broadcastArray = sc.broadcast(array)
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- array(0) = 5
- sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
- """)
+ test("running on Mesos") {
+ val output = runInterpreter("localquiet",
+ """
+ |var v = 7
+ |def getV() = v
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |v = 10
+ |sc.parallelize(1 to 10).map(x => getV()).collect.reduceLeft(_+_)
+ |var array = new Array[Int](5)
+ |val broadcastArray = sc.broadcast(array)
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ |array(0) = 5
+ |sc.parallelize(0 to 4).map(x => broadcastArray.value(x)).collect
+ """.stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains("res0: Int = 70", output)
diff --git a/bin/slaves.sh b/sbin/slaves.sh
index c367c2fd8e..a5bc2183d8 100755
--- a/bin/slaves.sh
+++ b/sbin/slaves.sh
@@ -36,10 +36,10 @@ if [ $# -le 0 ]; then
exit 1
fi
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
# If the slaves file is specified in the command line,
# then it takes precedence over the definition in
diff --git a/bin/spark-config.sh b/sbin/spark-config.sh
index cd2c7b7b0d..cd2c7b7b0d 100755
--- a/bin/spark-config.sh
+++ b/sbin/spark-config.sh
diff --git a/bin/spark-daemon.sh b/sbin/spark-daemon.sh
index a0c0d44b58..2be2b3d7c0 100755
--- a/bin/spark-daemon.sh
+++ b/sbin/spark-daemon.sh
@@ -37,10 +37,10 @@ if [ $# -le 1 ]; then
exit 1
fi
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
# get arguments
@@ -147,7 +147,7 @@ case $startStop in
spark_rotate_log "$log"
echo starting $command, logging to $log
cd "$SPARK_PREFIX"
- nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
+ nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null &
newpid=$!
echo $newpid > $pid
sleep 2
diff --git a/bin/spark-daemons.sh b/sbin/spark-daemons.sh
index 64286cb2da..5d9f2bb51c 100755
--- a/bin/spark-daemons.sh
+++ b/sbin/spark-daemons.sh
@@ -27,9 +27,9 @@ if [ $# -le 1 ]; then
exit 1
fi
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
-exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/spark-daemon.sh" "$@"
+exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/spark-daemon.sh" "$@"
diff --git a/spark-executor b/sbin/spark-executor
index 2c07c54843..de5bfab563 100755
--- a/spark-executor
+++ b/sbin/spark-executor
@@ -17,6 +17,7 @@
# limitations under the License.
#
-FWDIR="`dirname $0`"
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/spark-class org.apache.spark.executor.MesosExecutorBackend
+exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend
diff --git a/bin/start-all.sh b/sbin/start-all.sh
index 0182f1ab24..2daf49db35 100755
--- a/bin/start-all.sh
+++ b/sbin/start-all.sh
@@ -21,14 +21,14 @@
# Starts the master on this node.
# Starts a worker on each node specified in conf/slaves
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
# Load the Spark configuration
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
# Start Master
-"$bin"/start-master.sh
+"$sbin"/start-master.sh
# Start Workers
-"$bin"/start-slaves.sh
+"$sbin"/start-slaves.sh
diff --git a/bin/start-master.sh b/sbin/start-master.sh
index 648c7ae75f..3dcf7cc348 100755
--- a/bin/start-master.sh
+++ b/sbin/start-master.sh
@@ -19,10 +19,10 @@
# Starts the master on the machine this script is executed on.
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
. "${SPARK_CONF_DIR}/spark-env.sh"
@@ -49,4 +49,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
fi
fi
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT
diff --git a/bin/start-slave.sh b/sbin/start-slave.sh
index 4eefa20944..524be38c62 100755
--- a/bin/start-slave.sh
+++ b/sbin/start-slave.sh
@@ -20,8 +20,8 @@
# Usage: start-slave.sh <worker#> <master-spark-URL>
# where <master-spark-URL> is like "spark://localhost:7077"
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
# Set SPARK_PUBLIC_DNS so slaves can be linked in master web UI
if [ "$SPARK_PUBLIC_DNS" = "" ]; then
@@ -32,4 +32,4 @@ if [ "$SPARK_PUBLIC_DNS" = "" ]; then
fi
fi
-"$bin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
+"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@"
diff --git a/bin/start-slaves.sh b/sbin/start-slaves.sh
index 00dc4888b2..fd5cdeb1e6 100755
--- a/bin/start-slaves.sh
+++ b/sbin/start-slaves.sh
@@ -17,10 +17,10 @@
# limitations under the License.
#
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
. "${SPARK_CONF_DIR}/spark-env.sh"
@@ -37,12 +37,12 @@ fi
# Launch the slaves
if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
- exec "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
+ exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT
else
if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then
SPARK_WORKER_WEBUI_PORT=8081
fi
for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
- "$bin/slaves.sh" cd "$SPARK_HOME" \; "$bin/start-slave.sh" $(( $i + 1 )) spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
+ "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i ))
done
fi
diff --git a/bin/stop-all.sh b/sbin/stop-all.sh
index b6c83a7ba4..60b358d374 100755
--- a/bin/stop-all.sh
+++ b/sbin/stop-all.sh
@@ -21,12 +21,12 @@
# Run this on the master nde
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
# Load the Spark configuration
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
# Stop the slaves, then the master
-"$bin"/stop-slaves.sh
-"$bin"/stop-master.sh
+"$sbin"/stop-slaves.sh
+"$sbin"/stop-master.sh
diff --git a/bin/stop-master.sh b/sbin/stop-master.sh
index 310e33bedc..2adabd4265 100755
--- a/bin/stop-master.sh
+++ b/sbin/stop-master.sh
@@ -19,9 +19,9 @@
# Starts the master on the machine this script is executed on.
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
-"$bin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
+"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1
diff --git a/bin/stop-slaves.sh b/sbin/stop-slaves.sh
index fcb8555d4e..eb803b4900 100755
--- a/bin/stop-slaves.sh
+++ b/sbin/stop-slaves.sh
@@ -17,19 +17,19 @@
# limitations under the License.
#
-bin=`dirname "$0"`
-bin=`cd "$bin"; pwd`
+sbin=`dirname "$0"`
+sbin=`cd "$sbin"; pwd`
-. "$bin/spark-config.sh"
+. "$sbin/spark-config.sh"
if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
. "${SPARK_CONF_DIR}/spark-env.sh"
fi
if [ "$SPARK_WORKER_INSTANCES" = "" ]; then
- "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
+ "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1
else
for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do
- "$bin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
+ "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 ))
done
fi
diff --git a/sbt/sbt b/sbt/sbt
index c31a0280ff..7f47d90cf1 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -1,4 +1,4 @@
-#!/usr/bin/env bash
+#!/bin/bash
#
# Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,12 +17,45 @@
# limitations under the License.
#
-EXTRA_ARGS=""
-if [ "$MESOS_HOME" != "" ]; then
- EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java"
-fi
-
-export SPARK_HOME=$(cd "$(dirname $0)/.." 2>&1 >/dev/null ; pwd)
-export SPARK_TESTING=1 # To put test classes on classpath
+# This script launches sbt for this project. If present it uses the system
+# version of sbt. If there is no system version of sbt it attempts to download
+# sbt locally.
+SBT_VERSION=`awk -F "=" '/sbt\\.version/ {print $2}' ./project/build.properties`
+URL1=http://typesafe.artifactoryonline.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar
+URL2=http://repo.typesafe.com/typesafe/ivy-releases/org.scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch.jar
+JAR=sbt/sbt-launch-${SBT_VERSION}.jar
-java -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m $EXTRA_ARGS $SBT_OPTS -jar "$SPARK_HOME"/sbt/sbt-launch-*.jar "$@"
+printf "Checking for system sbt ["
+if hash sbt 2>/dev/null; then
+ printf "FOUND]\n"
+ # Use System SBT
+ sbt "$@"
+else
+ printf "NOT FOUND]\n"
+ # Download sbt or use already downloaded
+ if [ ! -d .sbtlib ]; then
+ mkdir .sbtlib
+ fi
+ if [ ! -f ${JAR} ]; then
+ # Download
+ printf "Attempting to fetch sbt\n"
+ if hash curl 2>/dev/null; then
+ curl --progress-bar ${URL1} > ${JAR} || curl --progress-bar ${URL2} > ${JAR}
+ elif hash wget 2>/dev/null; then
+ wget --progress=bar ${URL1} -O ${JAR} || wget --progress=bar ${URL2} -O ${JAR}
+ else
+ printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n"
+ exit -1
+ fi
+ fi
+ if [ ! -f ${JAR} ]; then
+ # We failed to download
+ printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n"
+ exit -1
+ fi
+ printf "Launching sbt from ${JAR}\n"
+ java \
+ -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \
+ -jar ${JAR} \
+ "$@"
+fi
diff --git a/sbt/sbt-launch-0.11.3-2.jar b/sbt/sbt-launch-0.11.3-2.jar
deleted file mode 100644
index 23e5c3f311..0000000000
--- a/sbt/sbt-launch-0.11.3-2.jar
+++ /dev/null
Binary files differ
diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd
deleted file mode 100644
index 681fe00f92..0000000000
--- a/sbt/sbt.cmd
+++ /dev/null
@@ -1,25 +0,0 @@
-@echo off
-
-rem
-rem Licensed to the Apache Software Foundation (ASF) under one or more
-rem contributor license agreements. See the NOTICE file distributed with
-rem this work for additional information regarding copyright ownership.
-rem The ASF licenses this file to You under the Apache License, Version 2.0
-rem (the "License"); you may not use this file except in compliance with
-rem the License. You may obtain a copy of the License at
-rem
-rem http://www.apache.org/licenses/LICENSE-2.0
-rem
-rem Unless required by applicable law or agreed to in writing, software
-rem distributed under the License is distributed on an "AS IS" BASIS,
-rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-rem See the License for the specific language governing permissions and
-rem limitations under the License.
-rem
-
-set EXTRA_ARGS=
-if not "%MESOS_HOME%x"=="x" set EXTRA_ARGS=-Djava.library.path=%MESOS_HOME%\lib\java
-
-set SPARK_HOME=%~dp0..
-
-java -Xmx1200M -XX:MaxPermSize=200m -XX:ReservedCodeCacheSize=256m %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-0.11.3-2.jar "%*"
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 7a9ae6a97b..459756912d 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -26,7 +26,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Streaming</name>
<url>http://spark.incubator.apache.org/</url>
@@ -48,7 +48,7 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -61,73 +61,17 @@
<version>1.9.11</version>
</dependency>
<dependency>
- <groupId>org.apache.kafka</groupId>
- <artifactId>kafka_2.9.2</artifactId>
- <version>0.8.0-beta1</version>
- <exclusions>
- <exclusion>
- <groupId>com.sun.jmx</groupId>
- <artifactId>jmxri</artifactId>
- </exclusion>
- <exclusion>
- <groupId>com.sun.jdmk</groupId>
- <artifactId>jmxtools</artifactId>
- </exclusion>
- <exclusion>
- <groupId>net.sf.jopt-simple</groupId>
- <artifactId>jopt-simple</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.flume</groupId>
- <artifactId>flume-ng-sdk</artifactId>
- <version>1.2.0</version>
- <exclusions>
- <exclusion>
- <groupId>org.jboss.netty</groupId>
- <artifactId>netty</artifactId>
- </exclusion>
- <exclusion>
- <groupId>org.xerial.snappy</groupId>
- <artifactId>*</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.twitter4j</groupId>
- <artifactId>twitter4j-stream</artifactId>
- <version>3.0.3</version>
- <exclusions>
- <exclusion>
- <groupId>org.jboss.netty</groupId>
- <artifactId>netty</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
</dependency>
<dependency>
- <groupId>com.typesafe.akka</groupId>
- <artifactId>akka-zeromq</artifactId>
- <version>2.0.3</version>
- <exclusions>
- <exclusion>
- <groupId>org.jboss.netty</groupId>
- <artifactId>netty</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
- <artifactId>scalacheck_2.9.3</artifactId>
+ <artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -141,19 +85,47 @@
<scope>test</scope>
</dependency>
<dependency>
- <groupId>org.eclipse.paho</groupId>
- <artifactId>mqtt-client</artifactId>
- <version>0.4.0</version>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
<artifactId>scalatest-maven-plugin</artifactId>
</plugin>
+
+ <!--
+ This plugin forces the generation of jar containing streaming test classes,
+ so that the tests classes of external modules can use them. The two execution profiles
+ are necessary - first one for 'mvn package', second one for 'mvn compile'. Ideally,
+ 'mvn compile' should not compile test classes and therefore should not need this.
+ However, an open Maven bug (http://jira.codehaus.org/browse/MNG-3559)
+ causes the compilation to fail if streaming test-jar is not generated. Hence, the
+ second execution profile for 'mvn compile'.
+ -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.2</version>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ <execution>
+ <id>test-jar-on-compile</id>
+ <phase>compile</phase>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
</plugins>
</build>
</project>
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index 9271914eb5..ca0115f90e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -21,12 +21,13 @@ import java.io._
import java.util.concurrent.Executors
import java.util.concurrent.RejectedExecutionException
-import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.conf.Configuration
-import org.apache.spark.Logging
+import org.apache.spark.{SparkConf, Logging}
import org.apache.spark.io.CompressionCodec
import org.apache.spark.util.MetadataCleaner
+import org.apache.spark.deploy.SparkHadoopUtil
private[streaming]
@@ -34,14 +35,14 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
extends Logging with Serializable {
val master = ssc.sc.master
val framework = ssc.sc.appName
- val sparkHome = ssc.sc.sparkHome
+ val sparkHome = ssc.sc.getSparkHome.getOrElse(null)
val jars = ssc.sc.jars
- val environment = ssc.sc.environment
val graph = ssc.graph
val checkpointDir = ssc.checkpointDir
val checkpointDuration = ssc.checkpointDuration
- val pendingTimes = ssc.scheduler.jobManager.getPendingTimes()
- val delaySeconds = MetadataCleaner.getDelaySeconds
+ val pendingTimes = ssc.scheduler.getPendingTimes()
+ val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf)
+ val sparkConf = ssc.conf
def validate() {
assert(master != null, "Checkpoint.master is null")
@@ -54,36 +55,36 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
/**
- * Convenience class to speed up the writing of graph checkpoint to file
+ * Convenience class to handle the writing of graph checkpoint to file
*/
private[streaming]
-class CheckpointWriter(checkpointDir: String) extends Logging {
+class CheckpointWriter(conf: SparkConf, checkpointDir: String, hadoopConf: Configuration)
+ extends Logging
+{
val file = new Path(checkpointDir, "graph")
- // The file to which we actually write - and then "move" to file.
- private val writeFile = new Path(file.getParent, file.getName + ".next")
- private val bakFile = new Path(file.getParent, file.getName + ".bk")
-
- private var stopped = false
-
- val conf = new Configuration()
- var fs = file.getFileSystem(conf)
- val maxAttempts = 3
+ val MAX_ATTEMPTS = 3
val executor = Executors.newFixedThreadPool(1)
+ val compressionCodec = CompressionCodec.createCodec(conf)
+ // The file to which we actually write - and then "move" to file
+ val writeFile = new Path(file.getParent, file.getName + ".next")
+ // The file to which existing checkpoint is backed up (i.e. "moved")
+ val bakFile = new Path(file.getParent, file.getName + ".bk")
- private val compressionCodec = CompressionCodec.createCodec()
+ private var stopped = false
+ private var fs_ : FileSystem = _
- // Removed code which validates whether there is only one CheckpointWriter per path 'file' since
+ // Removed code which validates whether there is only one CheckpointWriter per path 'file' since
// I did not notice any errors - reintroduce it ?
-
class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
def run() {
var attempts = 0
val startTime = System.currentTimeMillis()
- while (attempts < maxAttempts) {
+ while (attempts < MAX_ATTEMPTS && !stopped) {
attempts += 1
try {
logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'")
- // This is inherently thread unsafe .. so alleviating it by writing to '.new' and then doing moves : which should be pretty fast.
+ // This is inherently thread unsafe, so alleviating it by writing to '.new' and
+ // then moving it to the final file
val fos = fs.create(writeFile)
fos.write(bytes)
fos.close()
@@ -101,6 +102,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
} catch {
case ioe: IOException =>
logWarning("Error writing checkpoint to file in " + attempts + " attempts", ioe)
+ reset()
}
}
logError("Could not write checkpoint for time " + checkpointTime + " to file '" + file + "'")
@@ -133,7 +135,17 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
val startTime = System.currentTimeMillis()
val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS)
val endTime = System.currentTimeMillis()
- logInfo("CheckpointWriter executor terminated ? " + terminated + ", waited for " + (endTime - startTime) + " ms.")
+ logInfo("CheckpointWriter executor terminated ? " + terminated +
+ ", waited for " + (endTime - startTime) + " ms.")
+ }
+
+ private def fs = synchronized {
+ if (fs_ == null) fs_ = file.getFileSystem(hadoopConf)
+ fs_
+ }
+
+ private def reset() = synchronized {
+ fs_ = null
}
}
@@ -141,11 +153,12 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
private[streaming]
object CheckpointReader extends Logging {
- def read(path: String): Checkpoint = {
+ def read(conf: SparkConf, path: String): Checkpoint = {
val fs = new Path(path).getFileSystem(new Configuration())
- val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"), new Path(path), new Path(path + ".bk"))
+ val attempts = Seq(new Path(path, "graph"), new Path(path, "graph.bk"),
+ new Path(path), new Path(path + ".bk"))
- val compressionCodec = CompressionCodec.createCodec()
+ val compressionCodec = CompressionCodec.createCodec(conf)
attempts.foreach(file => {
if (fs.exists(file)) {
@@ -158,7 +171,8 @@ object CheckpointReader extends Logging {
// loader to find and load classes. This is a well know Java issue and has popped up
// in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627)
val zis = compressionCodec.compressedInputStream(fis)
- val ois = new ObjectInputStreamWithLoader(zis, Thread.currentThread().getContextClassLoader)
+ val ois = new ObjectInputStreamWithLoader(zis,
+ Thread.currentThread().getContextClassLoader)
val cp = ois.readObject.asInstanceOf[Checkpoint]
ois.close()
fs.close()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
index 9ceff754c4..00671ba520 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStream.scala
@@ -17,27 +17,23 @@
package org.apache.spark.streaming
-import org.apache.spark.streaming.dstream._
import StreamingContext._
-import org.apache.spark.util.MetadataCleaner
-
-//import Time._
-
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.streaming.scheduler.Job
import org.apache.spark.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.MetadataCleaner
-import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
+import scala.reflect.ClassTag
import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
-import org.apache.hadoop.fs.{FileSystem, Path}
-import org.apache.hadoop.conf.Configuration
/**
* A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
- * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.rdd.RDD]]
* for more details on RDDs). DStreams can either be created from live data (such as, data from
* HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
* such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -56,12 +52,10 @@ import org.apache.hadoop.conf.Configuration
* - A function that is used to generate an RDD after each time interval
*/
-abstract class DStream[T: ClassManifest] (
+abstract class DStream[T: ClassTag] (
@transient protected[streaming] var ssc: StreamingContext
) extends Serializable with Logging {
- initLogging()
-
// =======================================================================
// Methods that should be implemented by subclasses of DStream
// =======================================================================
@@ -82,7 +76,7 @@ abstract class DStream[T: ClassManifest] (
// RDDs generated, marked as protected[streaming] so that testsuites can access it
@transient
protected[streaming] var generatedRDDs = new HashMap[Time, RDD[T]] ()
-
+
// Time zero for the DStream
protected[streaming] var zeroTime: Time = null
@@ -212,7 +206,7 @@ abstract class DStream[T: ClassManifest] (
checkpointDuration + "). Please set it to higher than " + checkpointDuration + "."
)
- val metadataCleanerDelay = MetadataCleaner.getDelaySeconds
+ val metadataCleanerDelay = MetadataCleaner.getDelaySeconds(ssc.conf)
logInfo("metadataCleanupDelay = " + metadataCleanerDelay)
assert(
metadataCleanerDelay < 0 || rememberDuration.milliseconds < metadataCleanerDelay * 1000,
@@ -274,16 +268,16 @@ abstract class DStream[T: ClassManifest] (
/**
* Retrieve a precomputed RDD of this DStream, or computes the RDD. This is an internal
* method that should not be called directly.
- */
+ */
protected[streaming] def getOrCompute(time: Time): Option[RDD[T]] = {
// If this DStream was not initialized (i.e., zeroTime not set), then do it
// If RDD was already generated, then retrieve it from HashMap
generatedRDDs.get(time) match {
-
- // If an RDD was already generated and is being reused, then
+
+ // If an RDD was already generated and is being reused, then
// probably all RDDs in this DStream will be reused and hence should be cached
case Some(oldRDD) => Some(oldRDD)
-
+
// if RDD was not generated, and if the time is valid
// (based on sliding time of this DStream), then generate the RDD
case None => {
@@ -300,7 +294,7 @@ abstract class DStream[T: ClassManifest] (
}
generatedRDDs.put(time, newRDD)
Some(newRDD)
- case None =>
+ case None =>
None
}
} else {
@@ -344,7 +338,7 @@ abstract class DStream[T: ClassManifest] (
dependencies.foreach(_.clearOldMetadata(time))
}
- /* Adds metadata to the Stream while it is running.
+ /* Adds metadata to the Stream while it is running.
* This methd should be overwritten by sublcasses of InputDStream.
*/
protected[streaming] def addMetadata(metadata: Any) {
@@ -416,7 +410,7 @@ abstract class DStream[T: ClassManifest] (
// =======================================================================
/** Return a new DStream by applying a function to all elements of this DStream. */
- def map[U: ClassManifest](mapFunc: T => U): DStream[U] = {
+ def map[U: ClassTag](mapFunc: T => U): DStream[U] = {
new MappedDStream(this, context.sparkContext.clean(mapFunc))
}
@@ -424,7 +418,7 @@ abstract class DStream[T: ClassManifest] (
* Return a new DStream by applying a function to all elements of this DStream,
* and then flattening the results
*/
- def flatMap[U: ClassManifest](flatMapFunc: T => Traversable[U]): DStream[U] = {
+ def flatMap[U: ClassTag](flatMapFunc: T => Traversable[U]): DStream[U] = {
new FlatMappedDStream(this, context.sparkContext.clean(flatMapFunc))
}
@@ -450,7 +444,7 @@ abstract class DStream[T: ClassManifest] (
* of this DStream. Applying mapPartitions() to an RDD applies a function to each partition
* of the RDD.
*/
- def mapPartitions[U: ClassManifest](
+ def mapPartitions[U: ClassTag](
mapPartFunc: Iterator[T] => Iterator[U],
preservePartitioning: Boolean = false
): DStream[U] = {
@@ -497,16 +491,14 @@ abstract class DStream[T: ClassManifest] (
* 'this' DStream will be registered as an output stream and therefore materialized.
*/
def foreach(foreachFunc: (RDD[T], Time) => Unit) {
- val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
- ssc.registerOutputStream(newStream)
- newStream
+ ssc.registerOutputStream(new ForEachDStream(this, context.sparkContext.clean(foreachFunc)))
}
/**
* Return a new DStream in which each RDD is generated by applying a function
* on each RDD of 'this' DStream.
*/
- def transform[U: ClassManifest](transformFunc: RDD[T] => RDD[U]): DStream[U] = {
+ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = {
transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r)))
}
@@ -514,7 +506,7 @@ abstract class DStream[T: ClassManifest] (
* Return a new DStream in which each RDD is generated by applying a function
* on each RDD of 'this' DStream.
*/
- def transform[U: ClassManifest](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
+ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = {
//new TransformedDStream(this, context.sparkContext.clean(transformFunc))
val cleanedF = context.sparkContext.clean(transformFunc)
val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
@@ -528,7 +520,7 @@ abstract class DStream[T: ClassManifest] (
* Return a new DStream in which each RDD is generated by applying a function
* on each RDD of 'this' DStream and 'other' DStream.
*/
- def transformWith[U: ClassManifest, V: ClassManifest](
+ def transformWith[U: ClassTag, V: ClassTag](
other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V]
): DStream[V] = {
val cleanedF = ssc.sparkContext.clean(transformFunc)
@@ -539,7 +531,7 @@ abstract class DStream[T: ClassManifest] (
* Return a new DStream in which each RDD is generated by applying a function
* on each RDD of 'this' DStream and 'other' DStream.
*/
- def transformWith[U: ClassManifest, V: ClassManifest](
+ def transformWith[U: ClassTag, V: ClassTag](
other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V]
): DStream[V] = {
val cleanedF = ssc.sparkContext.clean(transformFunc)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
index 58a0da2870..3fd5d52403 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamCheckpointData.scala
@@ -20,13 +20,16 @@ package org.apache.spark.streaming
import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.conf.Configuration
+
import collection.mutable.HashMap
import org.apache.spark.Logging
+import scala.collection.mutable.HashMap
+import scala.reflect.ClassTag
private[streaming]
-class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T])
+class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T])
extends Serializable with Logging {
protected val data = new HashMap[Time, AnyRef]()
@@ -107,4 +110,3 @@ class DStreamCheckpointData[T: ClassManifest] (dstream: DStream[T])
"[\n" + checkpointFiles.size + " checkpoint files \n" + checkpointFiles.mkString("\n") + "\n]"
}
}
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
index b9a58fded6..a09b891956 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
@@ -21,9 +21,9 @@ import dstream.InputDStream
import java.io.{ObjectInputStream, IOException, ObjectOutputStream}
import collection.mutable.ArrayBuffer
import org.apache.spark.Logging
+import org.apache.spark.streaming.scheduler.Job
final private[streaming] class DStreamGraph extends Serializable with Logging {
- initLogging()
private val inputStreams = new ArrayBuffer[InputDStream[_]]()
private val outputStreams = new ArrayBuffer[DStream[_]]()
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala
deleted file mode 100644
index 5233129506..0000000000
--- a/streaming/src/main/scala/org/apache/spark/streaming/JobManager.scala
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.streaming
-
-import org.apache.spark.Logging
-import org.apache.spark.SparkEnv
-import java.util.concurrent.Executors
-import collection.mutable.HashMap
-import collection.mutable.ArrayBuffer
-
-
-private[streaming]
-class JobManager(ssc: StreamingContext, numThreads: Int = 1) extends Logging {
-
- class JobHandler(ssc: StreamingContext, job: Job) extends Runnable {
- def run() {
- SparkEnv.set(ssc.env)
- try {
- val timeTaken = job.run()
- logInfo("Total delay: %.5f s for job %s of time %s (execution: %.5f s)".format(
- (System.currentTimeMillis() - job.time.milliseconds) / 1000.0, job.id, job.time.milliseconds, timeTaken / 1000.0))
- } catch {
- case e: Exception =>
- logError("Running " + job + " failed", e)
- }
- clearJob(job)
- }
- }
-
- initLogging()
-
- val jobExecutor = Executors.newFixedThreadPool(numThreads)
- val jobs = new HashMap[Time, ArrayBuffer[Job]]
-
- def runJob(job: Job) {
- jobs.synchronized {
- jobs.getOrElseUpdate(job.time, new ArrayBuffer[Job]) += job
- }
- jobExecutor.execute(new JobHandler(ssc, job))
- logInfo("Added " + job + " to queue")
- }
-
- def stop() {
- jobExecutor.shutdown()
- }
-
- private def clearJob(job: Job) {
- var timeCleared = false
- val time = job.time
- jobs.synchronized {
- val jobsOfTime = jobs.get(time)
- if (jobsOfTime.isDefined) {
- jobsOfTime.get -= job
- if (jobsOfTime.get.isEmpty) {
- jobs -= time
- timeCleared = true
- }
- } else {
- throw new Exception("Job finished for time " + job.time +
- " but time does not exist in jobs")
- }
- }
- if (timeCleared) {
- ssc.scheduler.clearOldMetadata(time)
- }
- }
-
- def getPendingTimes(): Array[Time] = {
- jobs.synchronized {
- jobs.keySet.toArray
- }
- }
-}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
index 8c12fd11ef..56dbcbda23 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/PairDStreamFunctions.scala
@@ -18,16 +18,15 @@
package org.apache.spark.streaming
import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.streaming.dstream.{ReducedWindowedDStream, StateDStream}
-import org.apache.spark.streaming.dstream.{ShuffledDStream}
-import org.apache.spark.streaming.dstream.{MapValuedDStream, FlatMapValuedDStream}
+import org.apache.spark.streaming.dstream._
import org.apache.spark.{Partitioner, HashPartitioner}
import org.apache.spark.SparkContext._
-import org.apache.spark.rdd.{Manifests, RDD, PairRDDFunctions}
+import org.apache.spark.rdd.{ClassTags, RDD, PairRDDFunctions}
import org.apache.spark.storage.StorageLevel
import scala.collection.mutable.ArrayBuffer
+import scala.reflect.{ClassTag, classTag}
import org.apache.hadoop.mapred.{JobConf, OutputFormat}
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
@@ -35,7 +34,7 @@ import org.apache.hadoop.mapred.OutputFormat
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.conf.Configuration
-class PairDStreamFunctions[K: ClassManifest, V: ClassManifest](self: DStream[(K,V)])
+class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)])
extends Serializable {
private[streaming] def ssc = self.ssc
@@ -105,12 +104,13 @@ extends Serializable {
* combineByKey for RDDs. Please refer to combineByKey in
* [[org.apache.spark.rdd.PairRDDFunctions]] for more information.
*/
- def combineByKey[C: ClassManifest](
+ def combineByKey[C: ClassTag](
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiner: (C, C) => C,
- partitioner: Partitioner) : DStream[(K, C)] = {
- new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner)
+ partitioner: Partitioner,
+ mapSideCombine: Boolean = true): DStream[(K, C)] = {
+ new ShuffledDStream[K, V, C](self, createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine)
}
/**
@@ -174,7 +174,13 @@ extends Serializable {
slideDuration: Duration,
partitioner: Partitioner
): DStream[(K, Seq[V])] = {
- self.window(windowDuration, slideDuration).groupByKey(partitioner)
+ val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v
+ val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v
+ val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2
+ self.groupByKey(partitioner)
+ .window(windowDuration, slideDuration)
+ .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner)
+ .asInstanceOf[DStream[(K, Seq[V])]]
}
/**
@@ -205,7 +211,7 @@ extends Serializable {
* DStream's batching interval
*/
def reduceByKeyAndWindow(
- reduceFunc: (V, V) => V,
+ reduceFunc: (V, V) => V,
windowDuration: Duration,
slideDuration: Duration
): DStream[(K, V)] = {
@@ -336,7 +342,7 @@ extends Serializable {
* corresponding state key-value pair will be eliminated.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Seq[V], Option[S]) => Option[S]
): DStream[(K, S)] = {
updateStateByKey(updateFunc, defaultPartitioner())
@@ -351,7 +357,7 @@ extends Serializable {
* @param numPartitions Number of partitions of each RDD in the new DStream.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Seq[V], Option[S]) => Option[S],
numPartitions: Int
): DStream[(K, S)] = {
@@ -367,7 +373,7 @@ extends Serializable {
* @param partitioner Partitioner for controlling the partitioning of each RDD in the new DStream.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Seq[V], Option[S]) => Option[S],
partitioner: Partitioner
): DStream[(K, S)] = {
@@ -390,7 +396,7 @@ extends Serializable {
* @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs.
* @tparam S State type
*/
- def updateStateByKey[S: ClassManifest](
+ def updateStateByKey[S: ClassTag](
updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
partitioner: Partitioner,
rememberPartitioner: Boolean
@@ -402,7 +408,7 @@ extends Serializable {
* Return a new DStream by applying a map function to the value of each key-value pairs in
* 'this' DStream without changing the key.
*/
- def mapValues[U: ClassManifest](mapValuesFunc: V => U): DStream[(K, U)] = {
+ def mapValues[U: ClassTag](mapValuesFunc: V => U): DStream[(K, U)] = {
new MapValuedDStream[K, V, U](self, mapValuesFunc)
}
@@ -410,7 +416,7 @@ extends Serializable {
* Return a new DStream by applying a flatmap function to the value of each key-value pairs in
* 'this' DStream without changing the key.
*/
- def flatMapValues[U: ClassManifest](
+ def flatMapValues[U: ClassTag](
flatMapValuesFunc: V => TraversableOnce[U]
): DStream[(K, U)] = {
new FlatMapValuedDStream[K, V, U](self, flatMapValuesFunc)
@@ -421,7 +427,7 @@ extends Serializable {
* Hash partitioning is used to generate the RDDs with Spark's default number
* of partitions.
*/
- def cogroup[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
+ def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = {
cogroup(other, defaultPartitioner())
}
@@ -429,7 +435,7 @@ extends Serializable {
* Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
* Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
*/
- def cogroup[W: ClassManifest](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (Seq[V], Seq[W]))] = {
+ def cogroup[W: ClassTag](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (Seq[V], Seq[W]))] = {
cogroup(other, defaultPartitioner(numPartitions))
}
@@ -437,7 +443,7 @@ extends Serializable {
* Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream.
* The supplied [[org.apache.spark.Partitioner]] is used to partition the generated RDDs.
*/
- def cogroup[W: ClassManifest](
+ def cogroup[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (Seq[V], Seq[W]))] = {
@@ -451,7 +457,7 @@ extends Serializable {
* Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
* Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
*/
- def join[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
+ def join[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (V, W))] = {
join[W](other, defaultPartitioner())
}
@@ -459,7 +465,7 @@ extends Serializable {
* Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
* Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
*/
- def join[W: ClassManifest](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (V, W))] = {
+ def join[W: ClassTag](other: DStream[(K, W)], numPartitions: Int): DStream[(K, (V, W))] = {
join[W](other, defaultPartitioner(numPartitions))
}
@@ -467,7 +473,7 @@ extends Serializable {
* Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream.
* The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD.
*/
- def join[W: ClassManifest](
+ def join[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (V, W))] = {
@@ -482,7 +488,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
* number of partitions.
*/
- def leftOuterJoin[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (V, Option[W]))] = {
+ def leftOuterJoin[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (V, Option[W]))] = {
leftOuterJoin[W](other, defaultPartitioner())
}
@@ -491,7 +497,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
* partitions.
*/
- def leftOuterJoin[W: ClassManifest](
+ def leftOuterJoin[W: ClassTag](
other: DStream[(K, W)],
numPartitions: Int
): DStream[(K, (V, Option[W]))] = {
@@ -503,7 +509,7 @@ extends Serializable {
* `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control
* the partitioning of each RDD.
*/
- def leftOuterJoin[W: ClassManifest](
+ def leftOuterJoin[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (V, Option[W]))] = {
@@ -518,7 +524,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default
* number of partitions.
*/
- def rightOuterJoin[W: ClassManifest](other: DStream[(K, W)]): DStream[(K, (Option[V], W))] = {
+ def rightOuterJoin[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Option[V], W))] = {
rightOuterJoin[W](other, defaultPartitioner())
}
@@ -527,7 +533,7 @@ extends Serializable {
* `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions`
* partitions.
*/
- def rightOuterJoin[W: ClassManifest](
+ def rightOuterJoin[W: ClassTag](
other: DStream[(K, W)],
numPartitions: Int
): DStream[(K, (Option[V], W))] = {
@@ -539,7 +545,7 @@ extends Serializable {
* `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control
* the partitioning of each RDD.
*/
- def rightOuterJoin[W: ClassManifest](
+ def rightOuterJoin[W: ClassTag](
other: DStream[(K, W)],
partitioner: Partitioner
): DStream[(K, (Option[V], W))] = {
@@ -556,8 +562,8 @@ extends Serializable {
def saveAsHadoopFiles[F <: OutputFormat[K, V]](
prefix: String,
suffix: String
- )(implicit fm: ClassManifest[F]) {
- saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ )(implicit fm: ClassTag[F]) {
+ saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -586,8 +592,8 @@ extends Serializable {
def saveAsNewAPIHadoopFiles[F <: NewOutputFormat[K, V]](
prefix: String,
suffix: String
- )(implicit fm: ClassManifest[F]) {
- saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.erasure.asInstanceOf[Class[F]])
+ )(implicit fm: ClassTag[F]) {
+ saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]])
}
/**
@@ -609,9 +615,7 @@ extends Serializable {
self.foreach(saveFunc)
}
- private def getKeyClass() = implicitly[ClassManifest[K]].erasure
+ private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass
- private def getValueClass() = implicitly[ClassManifest[V]].erasure
+ private def getValueClass() = implicitly[ClassTag[V]].runtimeClass
}
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 70bf902143..693cb7fc30 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -17,36 +17,28 @@
package org.apache.spark.streaming
-import akka.actor.Props
-import akka.actor.SupervisorStrategy
-import akka.zeromq.Subscribe
-
-import org.apache.spark.streaming.dstream._
-
-import org.apache.spark._
-import org.apache.spark.rdd.RDD
-import org.apache.spark.streaming.receivers.ActorReceiver
-import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy
-import org.apache.spark.streaming.receivers.ZeroMQReceiver
-import org.apache.spark.storage.StorageLevel
-import org.apache.spark.util.MetadataCleaner
-import org.apache.spark.streaming.receivers.ActorReceiver
-
import scala.collection.mutable.Queue
import scala.collection.Map
+import scala.reflect.ClassTag
import java.io.InputStream
import java.util.concurrent.atomic.AtomicInteger
-import java.util.UUID
+import akka.actor.Props
+import akka.actor.SupervisorStrategy
import org.apache.hadoop.io.LongWritable
import org.apache.hadoop.io.Text
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat
import org.apache.hadoop.fs.Path
-import twitter4j.Status
-import twitter4j.auth.Authorization
+import org.apache.spark._
+import org.apache.spark.rdd.RDD
+import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.MetadataCleaner
+import org.apache.spark.streaming.dstream._
+import org.apache.spark.streaming.receivers._
+import org.apache.spark.streaming.scheduler._
/**
* A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -61,18 +53,27 @@ class StreamingContext private (
/**
* Create a StreamingContext using an existing SparkContext.
- * @param sparkContext Existing SparkContext
- * @param batchDuration The time interval at which streaming data will be divided into batches
+ * @param sparkContext existing SparkContext
+ * @param batchDuration the time interval at which streaming data will be divided into batches
*/
def this(sparkContext: SparkContext, batchDuration: Duration) = {
this(sparkContext, null, batchDuration)
}
/**
+ * Create a StreamingContext by providing the configuration necessary for a new SparkContext.
+ * @param conf a [[org.apache.spark.SparkConf]] object specifying Spark parameters
+ * @param batchDuration the time interval at which streaming data will be divided into batches
+ */
+ def this(conf: SparkConf, batchDuration: Duration) = {
+ this(StreamingContext.createNewSparkContext(conf), null, batchDuration)
+ }
+
+ /**
* Create a StreamingContext by providing the details necessary for creating a new SparkContext.
- * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param appName A name for your job, to display on the cluster web UI
- * @param batchDuration The time interval at which streaming data will be divided into batches
+ * @param master cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
+ * @param appName a name for your job, to display on the cluster web UI
+ * @param batchDuration the time interval at which streaming data will be divided into batches
*/
def this(
master: String,
@@ -85,26 +86,25 @@ class StreamingContext private (
null, batchDuration)
}
-
/**
* Re-create a StreamingContext from a checkpoint file.
* @param path Path either to the directory that was specified as the checkpoint directory, or
* to the checkpoint file 'graph' or 'graph.bk'.
*/
- def this(path: String) = this(null, CheckpointReader.read(path), null)
-
- initLogging()
+ def this(path: String) = this(null, CheckpointReader.read(new SparkConf(), path), null)
if (sc_ == null && cp_ == null) {
throw new Exception("Spark Streaming cannot be initialized with " +
"both SparkContext and checkpoint as null")
}
- if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds < 0) {
- MetadataCleaner.setDelaySeconds(cp_.delaySeconds)
+ private val conf_ = Option(sc_).map(_.conf).getOrElse(cp_.sparkConf)
+
+ if(cp_ != null && cp_.delaySeconds >= 0 && MetadataCleaner.getDelaySeconds(conf_) < 0) {
+ MetadataCleaner.setDelaySeconds(conf_, cp_.delaySeconds)
}
- if (MetadataCleaner.getDelaySeconds < 0) {
+ if (MetadataCleaner.getDelaySeconds(conf_) < 0) {
throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; "
+ "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)")
}
@@ -113,12 +113,14 @@ class StreamingContext private (
protected[streaming] val sc: SparkContext = {
if (isCheckpointPresent) {
- new SparkContext(cp_.master, cp_.framework, cp_.sparkHome, cp_.jars, cp_.environment)
+ new SparkContext(cp_.sparkConf)
} else {
sc_
}
}
+ protected[streaming] val conf = sc.conf
+
protected[streaming] val env = SparkEnv.get
protected[streaming] val graph: DStreamGraph = {
@@ -139,16 +141,17 @@ class StreamingContext private (
protected[streaming] var checkpointDir: String = {
if (isCheckpointPresent) {
- sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(cp_.checkpointDir), true)
+ sc.setCheckpointDir(cp_.checkpointDir)
cp_.checkpointDir
} else {
null
}
}
- protected[streaming] var checkpointDuration: Duration = if (isCheckpointPresent) cp_.checkpointDuration else null
- protected[streaming] var receiverJobThread: Thread = null
- protected[streaming] var scheduler: Scheduler = null
+ protected[streaming] val checkpointDuration: Duration = {
+ if (isCheckpointPresent) cp_.checkpointDuration else graph.batchDuration
+ }
+ protected[streaming] val scheduler = new JobScheduler(this)
/**
* Return the associated Spark context
@@ -173,8 +176,12 @@ class StreamingContext private (
*/
def checkpoint(directory: String) {
if (directory != null) {
- sc.setCheckpointDir(StreamingContext.getSparkCheckpointDir(directory))
- checkpointDir = directory
+ val path = new Path(directory)
+ val fs = path.getFileSystem(sparkContext.hadoopConfiguration)
+ fs.mkdirs(path)
+ val fullPath = fs.getFileStatus(path).getPath().toString
+ sc.setCheckpointDir(fullPath)
+ checkpointDir = fullPath
} else {
checkpointDir = null
}
@@ -191,7 +198,7 @@ class StreamingContext private (
* Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html
* @param receiver Custom implementation of NetworkReceiver
*/
- def networkStream[T: ClassManifest](
+ def networkStream[T: ClassTag](
receiver: NetworkReceiver[T]): DStream[T] = {
val inputStream = new PluggableInputDStream[T](this,
receiver)
@@ -211,7 +218,7 @@ class StreamingContext private (
* to ensure the type safety, i.e parametrized type of data received and actorStream
* should be same.
*/
- def actorStream[T: ClassManifest](
+ def actorStream[T: ClassTag](
props: Props,
name: String,
storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
@@ -221,74 +228,6 @@ class StreamingContext private (
}
/**
- * Create an input stream that receives messages pushed by a zeromq publisher.
- * @param publisherUrl Url of remote zeromq publisher
- * @param subscribe topic to subscribe to
- * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic
- * and each frame has sequence of byte thus it needs the converter
- * (which might be deserializer of bytes) to translate from sequence
- * of sequence of bytes, where sequence refer to a frame
- * and sub sequence refer to its payload.
- * @param storageLevel RDD storage level. Defaults to memory-only.
- */
- def zeroMQStream[T: ClassManifest](
- publisherUrl:String,
- subscribe: Subscribe,
- bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T],
- storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2,
- supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy
- ): DStream[T] = {
- actorStream(Props(new ZeroMQReceiver(publisherUrl,subscribe,bytesToObjects)),
- "ZeroMQReceiver", storageLevel, supervisorStrategy)
- }
-
- /**
- * Create an input stream that pulls messages from a Kafka Broker.
- * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
- * @param groupId The group id for this consumer.
- * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
- * in its own thread.
- * @param storageLevel Storage level to use for storing the received objects
- * (default: StorageLevel.MEMORY_AND_DISK_SER_2)
- */
- def kafkaStream(
- zkQuorum: String,
- groupId: String,
- topics: Map[String, Int],
- storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
- ): DStream[(String, String)] = {
- val kafkaParams = Map[String, String](
- "zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
- "zookeeper.connection.timeout.ms" -> "10000")
- kafkaStream[String, String, kafka.serializer.StringDecoder, kafka.serializer.StringDecoder](
- kafkaParams,
- topics,
- storageLevel)
- }
-
- /**
- * Create an input stream that pulls messages from a Kafka Broker.
- * @param kafkaParams Map of kafka configuration paramaters.
- * See: http://kafka.apache.org/configuration.html
- * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
- * in its own thread.
- * @param storageLevel Storage level to use for storing the received objects
- */
- def kafkaStream[
- K: ClassManifest,
- V: ClassManifest,
- U <: kafka.serializer.Decoder[_]: Manifest,
- T <: kafka.serializer.Decoder[_]: Manifest](
- kafkaParams: Map[String, String],
- topics: Map[String, Int],
- storageLevel: StorageLevel
- ): DStream[(K, V)] = {
- val inputStream = new KafkaInputDStream[K, V, U, T](this, kafkaParams, topics, storageLevel)
- registerInputStream(inputStream)
- inputStream
- }
-
- /**
* Create a input stream from TCP source hostname:port. Data is received using
* a TCP socket and the receive bytes is interpreted as UTF8 encoded `\n` delimited
* lines.
@@ -315,7 +254,7 @@ class StreamingContext private (
* @param storageLevel Storage level to use for storing the received objects
* @tparam T Type of the objects received (after converting bytes to objects)
*/
- def socketStream[T: ClassManifest](
+ def socketStream[T: ClassTag](
hostname: String,
port: Int,
converter: (InputStream) => Iterator[T],
@@ -327,22 +266,6 @@ class StreamingContext private (
}
/**
- * Create a input stream from a Flume source.
- * @param hostname Hostname of the slave machine to which the flume data will be sent
- * @param port Port of the slave machine to which the flume data will be sent
- * @param storageLevel Storage level to use for storing the received objects
- */
- def flumeStream (
- hostname: String,
- port: Int,
- storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
- ): DStream[SparkFlumeEvent] = {
- val inputStream = new FlumeInputDStream(this, hostname, port, storageLevel)
- registerInputStream(inputStream)
- inputStream
- }
-
- /**
* Create a input stream from network source hostname:port, where data is received
* as serialized blocks (serialized using the Spark's serializer) that can be directly
* pushed into the block manager without deserializing them. This is the most efficient
@@ -352,7 +275,7 @@ class StreamingContext private (
* @param storageLevel Storage level to use for storing the received objects
* @tparam T Type of the objects in the received blocks
*/
- def rawSocketStream[T: ClassManifest](
+ def rawSocketStream[T: ClassTag](
hostname: String,
port: Int,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
@@ -365,16 +288,17 @@ class StreamingContext private (
/**
* Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them using the given key-value types and input format.
- * File names starting with . are ignored.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
* @tparam K Key type for reading HDFS file
* @tparam V Value type for reading HDFS file
* @tparam F Input format for reading HDFS file
*/
def fileStream[
- K: ClassManifest,
- V: ClassManifest,
- F <: NewInputFormat[K, V]: ClassManifest
+ K: ClassTag,
+ V: ClassTag,
+ F <: NewInputFormat[K, V]: ClassTag
] (directory: String): DStream[(K, V)] = {
val inputStream = new FileInputDStream[K, V, F](this, directory)
registerInputStream(inputStream)
@@ -384,6 +308,8 @@ class StreamingContext private (
/**
* Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them using the given key-value types and input format.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system.
* @param directory HDFS directory to monitor for new file
* @param filter Function to filter paths to process
* @param newFilesOnly Should process only new files and ignore existing files in the directory
@@ -392,9 +318,9 @@ class StreamingContext private (
* @tparam F Input format for reading HDFS file
*/
def fileStream[
- K: ClassManifest,
- V: ClassManifest,
- F <: NewInputFormat[K, V]: ClassManifest
+ K: ClassTag,
+ V: ClassTag,
+ F <: NewInputFormat[K, V]: ClassTag
] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = {
val inputStream = new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly)
registerInputStream(inputStream)
@@ -404,7 +330,9 @@ class StreamingContext private (
/**
* Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them as text files (using key as LongWritable, value
- * as Text and input format as TextInputFormat). File names starting with . are ignored.
+ * as Text and input format as TextInputFormat). Files must be written to the
+ * monitored directory by "moving" them from another location within the same
+ * file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
*/
def textFileStream(directory: String): DStream[String] = {
@@ -412,31 +340,13 @@ class StreamingContext private (
}
/**
- * Create a input stream that returns tweets received from Twitter.
- * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth
- * authorization; this uses the system properties twitter4j.oauth.consumerKey,
- * .consumerSecret, .accessToken and .accessTokenSecret.
- * @param filters Set of filter strings to get only those tweets that match them
- * @param storageLevel Storage level to use for storing the received objects
- */
- def twitterStream(
- twitterAuth: Option[Authorization] = None,
- filters: Seq[String] = Nil,
- storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
- ): DStream[Status] = {
- val inputStream = new TwitterInputDStream(this, twitterAuth, filters, storageLevel)
- registerInputStream(inputStream)
- inputStream
- }
-
- /**
* Create an input stream from a queue of RDDs. In each batch,
* it will process either one or all of the RDDs returned by the queue.
* @param queue Queue of RDDs
* @param oneAtATime Whether only one RDD should be consumed from the queue in every interval
* @tparam T Type of objects in the RDD
*/
- def queueStream[T: ClassManifest](
+ def queueStream[T: ClassTag](
queue: Queue[RDD[T]],
oneAtATime: Boolean = true
): DStream[T] = {
@@ -452,7 +362,7 @@ class StreamingContext private (
* Set as null if no RDD should be returned when empty
* @tparam T Type of objects in the RDD
*/
- def queueStream[T: ClassManifest](
+ def queueStream[T: ClassTag](
queue: Queue[RDD[T]],
oneAtATime: Boolean,
defaultRDD: RDD[T]
@@ -462,25 +372,10 @@ class StreamingContext private (
inputStream
}
-/**
- * Create an input stream that receives messages pushed by a mqtt publisher.
- * @param brokerUrl Url of remote mqtt publisher
- * @param topic topic name to subscribe to
- * @param storageLevel RDD storage level. Defaults to memory-only.
- */
-
- def mqttStream(
- brokerUrl: String,
- topic: String,
- storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2): DStream[String] = {
- val inputStream = new MQTTInputDStream[String](this, brokerUrl, topic, storageLevel)
- registerInputStream(inputStream)
- inputStream
- }
/**
* Create a unified DStream from multiple DStreams of the same type and same slide duration.
*/
- def union[T: ClassManifest](streams: Seq[DStream[T]]): DStream[T] = {
+ def union[T: ClassTag](streams: Seq[DStream[T]]): DStream[T] = {
new UnionDStream[T](streams.toArray)
}
@@ -488,7 +383,7 @@ class StreamingContext private (
* Create a new DStream in which each RDD is generated by applying a function on RDDs of
* the DStreams.
*/
- def transform[T: ClassManifest](
+ def transform[T: ClassTag](
dstreams: Seq[DStream[_]],
transformFunc: (Seq[RDD[_]], Time) => RDD[T]
): DStream[T] = {
@@ -510,6 +405,13 @@ class StreamingContext private (
graph.addOutputStream(outputStream)
}
+ /** Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for
+ * receiving system events related to streaming.
+ */
+ def addStreamingListener(streamingListener: StreamingListener) {
+ scheduler.listenerBus.addListener(streamingListener)
+ }
+
protected def validate() {
assert(graph != null, "Graph is null")
graph.validate()
@@ -525,27 +427,22 @@ class StreamingContext private (
* Start the execution of the streams.
*/
def start() {
- if (checkpointDir != null && checkpointDuration == null && graph != null) {
- checkpointDuration = graph.batchDuration
- }
-
validate()
+ // Get the network input streams
val networkInputStreams = graph.getInputStreams().filter(s => s match {
case n: NetworkInputDStream[_] => true
case _ => false
}).map(_.asInstanceOf[NetworkInputDStream[_]]).toArray
+ // Start the network input tracker (must start before receivers)
if (networkInputStreams.length > 0) {
- // Start the network input tracker (must start before receivers)
networkInputTracker = new NetworkInputTracker(this, networkInputStreams)
networkInputTracker.start()
}
-
Thread.sleep(1000)
// Start the scheduler
- scheduler = new Scheduler(this)
scheduler.start()
}
@@ -556,7 +453,6 @@ class StreamingContext private (
try {
if (scheduler != null) scheduler.stop()
if (networkInputTracker != null) networkInputTracker.stop()
- if (receiverJobThread != null) receiverJobThread.interrupt()
sc.stop()
logInfo("StreamingContext stopped successfully")
} catch {
@@ -568,22 +464,40 @@ class StreamingContext private (
object StreamingContext {
- implicit def toPairDStreamFunctions[K: ClassManifest, V: ClassManifest](stream: DStream[(K,V)]) = {
+ implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = {
new PairDStreamFunctions[K, V](stream)
}
+ /**
+ * Find the JAR from which a given class was loaded, to make it easy for users to pass
+ * their JARs to SparkContext.
+ */
+ def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls)
+
+ protected[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = {
+ // Set the default cleaner delay to an hour if not already set.
+ // This should be sufficient for even 1 second batch intervals.
+ val sc = new SparkContext(conf)
+ if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) {
+ MetadataCleaner.setDelaySeconds(sc.conf, 3600)
+ }
+ sc
+ }
+
protected[streaming] def createNewSparkContext(
master: String,
appName: String,
sparkHome: String,
jars: Seq[String],
- environment: Map[String, String]): SparkContext = {
+ environment: Map[String, String]): SparkContext =
+ {
+ val sc = new SparkContext(master, appName, sparkHome, jars, environment)
// Set the default cleaner delay to an hour if not already set.
- // This should be sufficient for even 1 second interval.
- if (MetadataCleaner.getDelaySeconds < 0) {
- MetadataCleaner.setDelaySeconds(3600)
+ // This should be sufficient for even 1 second batch intervals.
+ if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) {
+ MetadataCleaner.setDelaySeconds(sc.conf, 3600)
}
- new SparkContext(master, appName, sparkHome, jars, environment)
+ sc
}
protected[streaming] def rddToFileName[T](prefix: String, suffix: String, time: Time): String = {
@@ -595,8 +509,4 @@ object StreamingContext {
prefix + "-" + time.milliseconds + "." + suffix
}
}
-
- protected[streaming] def getSparkCheckpointDir(sscCheckpointDir: String): String = {
- new Path(sscCheckpointDir, UUID.randomUUID.toString).toString
- }
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
index 1a2aeaa879..d29033df32 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala
@@ -23,9 +23,11 @@ import org.apache.spark.api.java.JavaRDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
+
/**
* A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
- * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.RDD]]
+ * sequence of RDDs (of the same type) representing a continuous stream of data (see [[org.apache.spark.rdd.RDD]]
* for more details on RDDs). DStreams can either be created from live data (such as, data from
* HDFS, Kafka or Flume) or it can be generated by transformation existing DStreams using operations
* such as `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each
@@ -41,7 +43,7 @@ import org.apache.spark.rdd.RDD
* - A time interval at which the DStream generates an RDD
* - A function that is used to generate an RDD after each time interval
*/
-class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassManifest[T])
+class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T])
extends JavaDStreamLike[T, JavaDStream[T], JavaRDD[T]] {
override def wrapRDD(rdd: RDD[T]): JavaRDD[T] = JavaRDD.fromRDD(rdd)
@@ -103,6 +105,6 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classManifest: ClassM
}
object JavaDStream {
- implicit def fromDStream[T: ClassManifest](dstream: DStream[T]): JavaDStream[T] =
+ implicit def fromDStream[T: ClassTag](dstream: DStream[T]): JavaDStream[T] =
new JavaDStream[T](dstream)
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
index 09189eadd8..64f38ce1c0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
@@ -21,6 +21,7 @@ import java.util.{List => JList}
import java.lang.{Long => JLong}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.spark.streaming._
import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD}
@@ -32,7 +33,7 @@ import JavaDStream._
trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]]
extends Serializable {
- implicit val classManifest: ClassManifest[T]
+ implicit val classTag: ClassTag[T]
def dstream: DStream[T]
@@ -136,7 +137,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
/** Return a new DStream by applying a function to all elements of this DStream. */
def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairDStream[K2, V2] = {
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
new JavaPairDStream(dstream.map(f)(cm))(f.keyType(), f.valueType())
}
@@ -157,7 +158,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = {
import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala
- def cm = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K2, V2]]]
+ def cm = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K2, V2]]]
new JavaPairDStream(dstream.flatMap(fn)(cm))(f.keyType(), f.valueType())
}
@@ -260,8 +261,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
* on each RDD of 'this' DStream.
*/
def transform[U](transformFunc: JFunction[R, JavaRDD[U]]): JavaDStream[U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
def scalaTransform (in: RDD[T]): RDD[U] =
transformFunc.call(wrapRDD(in)).rdd
dstream.transform(scalaTransform(_))
@@ -272,8 +273,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
* on each RDD of 'this' DStream.
*/
def transform[U](transformFunc: JFunction2[R, Time, JavaRDD[U]]): JavaDStream[U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
def scalaTransform (in: RDD[T], time: Time): RDD[U] =
transformFunc.call(wrapRDD(in), time).rdd
dstream.transform(scalaTransform(_, _))
@@ -285,10 +286,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
*/
def transform[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
- implicit val cmk: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+ implicit val cmk: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
def scalaTransform (in: RDD[T]): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(in)).rdd
dstream.transform(scalaTransform(_))
@@ -300,10 +301,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
*/
def transform[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]):
JavaPairDStream[K2, V2] = {
- implicit val cmk: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+ implicit val cmk: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
def scalaTransform (in: RDD[T], time: Time): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(in), time).rdd
dstream.transform(scalaTransform(_, _))
@@ -317,10 +318,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaDStream[U],
transformFunc: JFunction3[R, JavaRDD[U], Time, JavaRDD[W]]
): JavaDStream[W] = {
- implicit val cmu: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
- implicit val cmv: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cmu: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
+ implicit val cmv: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[W] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[U, W](other.dstream, scalaTransform(_, _, _))
@@ -334,12 +335,12 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaDStream[U],
transformFunc: JFunction3[R, JavaRDD[U], Time, JavaPairRDD[K2, V2]]
): JavaPairDStream[K2, V2] = {
- implicit val cmu: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
- implicit val cmk2: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv2: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
+ implicit val cmu: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
+ implicit val cmk2: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv2: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[(K2, V2)] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[U, (K2, V2)](other.dstream, scalaTransform(_, _, _))
@@ -353,12 +354,12 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaPairDStream[K2, V2],
transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaRDD[W]]
): JavaDStream[W] = {
- implicit val cmk2: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv2: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
- implicit val cmw: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cmk2: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv2: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
+ implicit val cmw: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[(K2, V2), W](other.dstream, scalaTransform(_, _, _))
@@ -372,14 +373,14 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
other: JavaPairDStream[K2, V2],
transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaPairRDD[K3, V3]]
): JavaPairDStream[K3, V3] = {
- implicit val cmk2: ClassManifest[K2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K2]]
- implicit val cmv2: ClassManifest[V2] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V2]]
- implicit val cmk3: ClassManifest[K3] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K3]]
- implicit val cmv3: ClassManifest[V3] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V3]]
+ implicit val cmk2: ClassTag[K2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]]
+ implicit val cmv2: ClassTag[V2] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]]
+ implicit val cmk3: ClassTag[K3] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K3]]
+ implicit val cmv3: ClassTag[V3] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V3]]
def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[(K3, V3)] =
transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd
dstream.transformWith[(K2, V2), (K3, V3)](other.dstream, scalaTransform(_, _, _))
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index c6cd635afa..6c3467d405 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -21,6 +21,7 @@ import java.util.{List => JList}
import java.lang.{Long => JLong}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
@@ -36,8 +37,8 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.PairRDDFunctions
class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
- implicit val kManifest: ClassManifest[K],
- implicit val vManifest: ClassManifest[V])
+ implicit val kManifest: ClassTag[K],
+ implicit val vManifest: ClassTag[V])
extends JavaDStreamLike[(K, V), JavaPairDStream[K, V], JavaPairRDD[K, V]] {
override def wrapRDD(rdd: RDD[(K, V)]): JavaPairRDD[K, V] = JavaPairRDD.fromRDD(rdd)
@@ -154,7 +155,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
/**
* Combine elements of each key in DStream's RDDs using custom function. This is similar to the
- * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.PairRDDFunctions]] for more
+ * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
* information.
*/
def combineByKey[C](createCombiner: JFunction[V, C],
@@ -162,12 +163,28 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
mergeCombiners: JFunction2[C, C, C],
partitioner: Partitioner
): JavaPairDStream[K, C] = {
- implicit val cm: ClassManifest[C] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[C]]
+ implicit val cm: ClassTag[C] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner)
}
/**
+ * Combine elements of each key in DStream's RDDs using custom function. This is similar to the
+ * combineByKey for RDDs. Please refer to combineByKey in [[org.apache.spark.rdd.PairRDDFunctions]] for more
+ * information.
+ */
+ def combineByKey[C](createCombiner: JFunction[V, C],
+ mergeValue: JFunction2[C, V, C],
+ mergeCombiners: JFunction2[C, C, C],
+ partitioner: Partitioner,
+ mapSideCombine: Boolean
+ ): JavaPairDStream[K, C] = {
+ implicit val cm: ClassTag[C] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]]
+ dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine)
+ }
+
+ /**
* Return a new DStream by applying `groupByKey` over a sliding window. This is similar to
* `DStream.groupByKey()` but applies it over a sliding window. The new DStream generates RDDs
* with the same interval as this DStream. Hash partitioning is used to generate the RDDs with
@@ -428,8 +445,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
*/
def updateStateByKey[S](updateFunc: JFunction2[JList[V], Optional[S], Optional[S]])
: JavaPairDStream[K, S] = {
- implicit val cm: ClassManifest[S] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
+ implicit val cm: ClassTag[S] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]]
dstream.updateStateByKey(convertUpdateStateFunction(updateFunc))
}
@@ -446,8 +463,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
numPartitions: Int)
: JavaPairDStream[K, S] = {
- implicit val cm: ClassManifest[S] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
+ implicit val cm: ClassTag[S] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]]
dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), numPartitions)
}
@@ -464,8 +481,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
updateFunc: JFunction2[JList[V], Optional[S], Optional[S]],
partitioner: Partitioner
): JavaPairDStream[K, S] = {
- implicit val cm: ClassManifest[S] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[S]]
+ implicit val cm: ClassTag[S] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]]
dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), partitioner)
}
@@ -475,8 +492,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* 'this' DStream without changing the key.
*/
def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = {
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
dstream.mapValues(f)
}
@@ -487,8 +504,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = {
import scala.collection.JavaConverters._
def fn = (x: V) => f.apply(x).asScala
- implicit val cm: ClassManifest[U] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
+ implicit val cm: ClassTag[U] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]]
dstream.flatMapValues(fn)
}
@@ -498,8 +515,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* of partitions.
*/
def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
}
@@ -511,8 +528,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
numPartitions: Int
): JavaPairDStream[K, (JList[V], JList[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.cogroup(other.dstream, numPartitions)
.mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
}
@@ -525,8 +542,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
partitioner: Partitioner
): JavaPairDStream[K, (JList[V], JList[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.cogroup(other.dstream, partitioner)
.mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2))))
}
@@ -536,8 +553,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* Hash partitioning is used to generate the RDDs with Spark's default number of partitions.
*/
def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.join(other.dstream)
}
@@ -546,8 +563,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* Hash partitioning is used to generate the RDDs with `numPartitions` partitions.
*/
def join[W](other: JavaPairDStream[K, W], numPartitions: Int): JavaPairDStream[K, (V, W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.join(other.dstream, numPartitions)
}
@@ -559,8 +576,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
partitioner: Partitioner
): JavaPairDStream[K, (V, W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
dstream.join(other.dstream, partitioner)
}
@@ -570,8 +587,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* number of partitions.
*/
def leftOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, Optional[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.leftOuterJoin(other.dstream)
joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
}
@@ -585,8 +602,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
numPartitions: Int
): JavaPairDStream[K, (V, Optional[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.leftOuterJoin(other.dstream, numPartitions)
joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
}
@@ -599,8 +616,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
partitioner: Partitioner
): JavaPairDStream[K, (V, Optional[W])] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.leftOuterJoin(other.dstream, partitioner)
joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))}
}
@@ -611,8 +628,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* number of partitions.
*/
def rightOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (Optional[V], W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.rightOuterJoin(other.dstream)
joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
}
@@ -626,8 +643,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
numPartitions: Int
): JavaPairDStream[K, (Optional[V], W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.rightOuterJoin(other.dstream, numPartitions)
joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
}
@@ -641,8 +658,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
other: JavaPairDStream[K, W],
partitioner: Partitioner
): JavaPairDStream[K, (Optional[V], W)] = {
- implicit val cm: ClassManifest[W] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[W]]
+ implicit val cm: ClassTag[W] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]]
val joinResult = dstream.rightOuterJoin(other.dstream, partitioner)
joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}
}
@@ -722,24 +739,24 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
new JavaDStream[(K, V)](dstream)
}
- override val classManifest: ClassManifest[(K, V)] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[Tuple2[K, V]]]
+ override val classTag: ClassTag[(K, V)] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[K, V]]]
}
object JavaPairDStream {
- implicit def fromPairDStream[K: ClassManifest, V: ClassManifest](dstream: DStream[(K, V)]) = {
+ implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) = {
new JavaPairDStream[K, V](dstream)
}
def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = {
- implicit val cmk: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val cmv: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cmk: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val cmv: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
new JavaPairDStream[K, V](dstream.dstream)
}
- def scalaToJavaLong[K: ClassManifest](dstream: JavaPairDStream[K, Long])
+ def scalaToJavaLong[K: ClassTag](dstream: JavaPairDStream[K, Long])
: JavaPairDStream[K, JLong] = {
StreamingContext.toPairDStreamFunctions(dstream.dstream).mapValues(new JLong(_))
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index 7f9dab0ef9..7068f32517 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -17,25 +17,24 @@
package org.apache.spark.streaming.api.java
-import java.lang.{Integer => JInt}
-import java.io.InputStream
-import java.util.{Map => JMap, List => JList}
import scala.collection.JavaConversions._
+import scala.reflect.ClassTag
+
+import java.io.InputStream
+import java.lang.{Integer => JInt}
+import java.util.{List => JList, Map => JMap}
+import akka.actor.{Props, SupervisorStrategy}
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
-import twitter4j.Status
-import akka.actor.Props
-import akka.actor.SupervisorStrategy
-import akka.zeromq.Subscribe
-import twitter4j.auth.Authorization
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext}
+import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2}
-import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaRDD}
import org.apache.spark.streaming._
-import org.apache.spark.streaming.dstream._
+import org.apache.spark.streaming.scheduler.StreamingListener
/**
* A StreamingContext is the main entry point for Spark Streaming functionality. Besides the basic
@@ -120,6 +119,14 @@ class JavaStreamingContext(val ssc: StreamingContext) {
this(new StreamingContext(sparkContext.sc, batchDuration))
/**
+ * Creates a StreamingContext using an existing SparkContext.
+ * @param conf A Spark application configuration
+ * @param batchDuration The time interval at which streaming data will be divided into batches
+ */
+ def this(conf: SparkConf, batchDuration: Duration) =
+ this(new StreamingContext(conf, batchDuration))
+
+ /**
* Re-creates a StreamingContext from a checkpoint file.
* @param path Path either to the directory that was specified as the checkpoint directory, or
* to the checkpoint file 'graph' or 'graph.bk'.
@@ -130,80 +137,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
val sc: JavaSparkContext = new JavaSparkContext(ssc.sc)
/**
- * Create an input stream that pulls messages form a Kafka Broker.
- * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
- * @param groupId The group id for this consumer.
- * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
- * in its own thread.
- */
- def kafkaStream(
- zkQuorum: String,
- groupId: String,
- topics: JMap[String, JInt])
- : JavaPairDStream[String, String] = {
- implicit val cmt: ClassManifest[String] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]]
- ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
- StorageLevel.MEMORY_ONLY_SER_2)
- }
-
- /**
- * Create an input stream that pulls messages form a Kafka Broker.
- * @param zkQuorum Zookeper quorum (hostname:port,hostname:port,..).
- * @param groupId The group id for this consumer.
- * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
- * in its own thread.
- * @param storageLevel RDD storage level. Defaults to memory-only
- *
- */
- def kafkaStream(
- zkQuorum: String,
- groupId: String,
- topics: JMap[String, JInt],
- storageLevel: StorageLevel)
- : JavaPairDStream[String, String] = {
- implicit val cmt: ClassManifest[String] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[String]]
- ssc.kafkaStream(zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
- storageLevel)
- }
-
- /**
- * Create an input stream that pulls messages form a Kafka Broker.
- * @param keyTypeClass Key type of RDD
- * @param valueTypeClass value type of RDD
- * @param keyDecoderClass Type of kafka key decoder
- * @param valueDecoderClass Type of kafka value decoder
- * @param kafkaParams Map of kafka configuration paramaters.
- * See: http://kafka.apache.org/configuration.html
- * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed
- * in its own thread.
- * @param storageLevel RDD storage level. Defaults to memory-only
- */
- def kafkaStream[K, V, U <: kafka.serializer.Decoder[_], T <: kafka.serializer.Decoder[_]](
- keyTypeClass: Class[K],
- valueTypeClass: Class[V],
- keyDecoderClass: Class[U],
- valueDecoderClass: Class[T],
- kafkaParams: JMap[String, String],
- topics: JMap[String, JInt],
- storageLevel: StorageLevel)
- : JavaPairDStream[K, V] = {
- implicit val keyCmt: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val valueCmt: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
-
- implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]]
- implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]]
-
- ssc.kafkaStream[K, V, U, T](
- kafkaParams.toMap,
- Map(topics.mapValues(_.intValue()).toSeq: _*),
- storageLevel)
- }
-
- /**
* Create a input stream from network source hostname:port. Data is received using
* a TCP socket and the receive bytes is interpreted as UTF8 encoded \n delimited
* lines.
@@ -245,15 +178,17 @@ class JavaStreamingContext(val ssc: StreamingContext) {
storageLevel: StorageLevel)
: JavaDStream[T] = {
def fn = (x: InputStream) => converter.apply(x).toIterator
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.socketStream(hostname, port, fn, storageLevel)
}
/**
- * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them as text files (using key as LongWritable, value
- * as Text and input format as TextInputFormat). File names starting with . are ignored.
+ * as Text and input format as TextInputFormat). Files must be written to the
+ * monitored directory by "moving" them from another location within the same
+ * file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
*/
def textFileStream(directory: String): JavaDStream[String] = {
@@ -274,8 +209,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
hostname: String,
port: Int,
storageLevel: StorageLevel): JavaDStream[T] = {
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel))
}
@@ -289,123 +224,32 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @tparam T Type of the objects in the received blocks
*/
def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = {
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port))
}
/**
- * Creates a input stream that monitors a Hadoop-compatible filesystem
+ * Create a input stream that monitors a Hadoop-compatible filesystem
* for new files and reads them using the given key-value types and input format.
- * File names starting with . are ignored.
+ * Files must be written to the monitored directory by "moving" them from another
+ * location within the same file system. File names starting with . are ignored.
* @param directory HDFS directory to monitor for new file
* @tparam K Key type for reading HDFS file
* @tparam V Value type for reading HDFS file
* @tparam F Input format for reading HDFS file
*/
def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = {
- implicit val cmk: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val cmv: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
- implicit val cmf: ClassManifest[F] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[F]]
+ implicit val cmk: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val cmv: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
+ implicit val cmf: ClassTag[F] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[F]]
ssc.fileStream[K, V, F](directory)
}
/**
- * Creates a input stream from a Flume source.
- * @param hostname Hostname of the slave machine to which the flume data will be sent
- * @param port Port of the slave machine to which the flume data will be sent
- * @param storageLevel Storage level to use for storing the received objects
- */
- def flumeStream(hostname: String, port: Int, storageLevel: StorageLevel):
- JavaDStream[SparkFlumeEvent] = {
- ssc.flumeStream(hostname, port, storageLevel)
- }
-
-
- /**
- * Creates a input stream from a Flume source.
- * @param hostname Hostname of the slave machine to which the flume data will be sent
- * @param port Port of the slave machine to which the flume data will be sent
- */
- def flumeStream(hostname: String, port: Int): JavaDStream[SparkFlumeEvent] = {
- ssc.flumeStream(hostname, port)
- }
-
- /**
- * Create a input stream that returns tweets received from Twitter.
- * @param twitterAuth Twitter4J Authorization object
- * @param filters Set of filter strings to get only those tweets that match them
- * @param storageLevel Storage level to use for storing the received objects
- */
- def twitterStream(
- twitterAuth: Authorization,
- filters: Array[String],
- storageLevel: StorageLevel
- ): JavaDStream[Status] = {
- ssc.twitterStream(Some(twitterAuth), filters, storageLevel)
- }
-
- /**
- * Create a input stream that returns tweets received from Twitter using Twitter4J's default
- * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
- * .consumerSecret, .accessToken and .accessTokenSecret to be set.
- * @param filters Set of filter strings to get only those tweets that match them
- * @param storageLevel Storage level to use for storing the received objects
- */
- def twitterStream(
- filters: Array[String],
- storageLevel: StorageLevel
- ): JavaDStream[Status] = {
- ssc.twitterStream(None, filters, storageLevel)
- }
-
- /**
- * Create a input stream that returns tweets received from Twitter.
- * @param twitterAuth Twitter4J Authorization
- * @param filters Set of filter strings to get only those tweets that match them
- */
- def twitterStream(
- twitterAuth: Authorization,
- filters: Array[String]
- ): JavaDStream[Status] = {
- ssc.twitterStream(Some(twitterAuth), filters)
- }
-
- /**
- * Create a input stream that returns tweets received from Twitter using Twitter4J's default
- * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
- * .consumerSecret, .accessToken and .accessTokenSecret to be set.
- * @param filters Set of filter strings to get only those tweets that match them
- */
- def twitterStream(
- filters: Array[String]
- ): JavaDStream[Status] = {
- ssc.twitterStream(None, filters)
- }
-
- /**
- * Create a input stream that returns tweets received from Twitter.
- * @param twitterAuth Twitter4J Authorization
- */
- def twitterStream(
- twitterAuth: Authorization
- ): JavaDStream[Status] = {
- ssc.twitterStream(Some(twitterAuth))
- }
-
- /**
- * Create a input stream that returns tweets received from Twitter using Twitter4J's default
- * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey,
- * .consumerSecret, .accessToken and .accessTokenSecret to be set.
- */
- def twitterStream(): JavaDStream[Status] = {
- ssc.twitterStream()
- }
-
- /**
* Create an input stream with any arbitrary user implemented actor receiver.
* @param props Props object defining creation of the actor
* @param name Name of the actor
@@ -422,8 +266,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
storageLevel: StorageLevel,
supervisorStrategy: SupervisorStrategy
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.actorStream[T](props, name, storageLevel, supervisorStrategy)
}
@@ -443,8 +287,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
name: String,
storageLevel: StorageLevel
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.actorStream[T](props, name, storageLevel)
}
@@ -462,76 +306,12 @@ class JavaStreamingContext(val ssc: StreamingContext) {
props: Props,
name: String
): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
ssc.actorStream[T](props, name)
}
/**
- * Create an input stream that receives messages pushed by a zeromq publisher.
- * @param publisherUrl Url of remote zeromq publisher
- * @param subscribe topic to subscribe to
- * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
- * of byte thus it needs the converter(which might be deserializer of bytes)
- * to translate from sequence of sequence of bytes, where sequence refer to a frame
- * and sub sequence refer to its payload.
- * @param storageLevel Storage level to use for storing the received objects
- */
- def zeroMQStream[T](
- publisherUrl:String,
- subscribe: Subscribe,
- bytesToObjects: Seq[Seq[Byte]] ⇒ Iterator[T],
- storageLevel: StorageLevel,
- supervisorStrategy: SupervisorStrategy
- ): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
- ssc.zeroMQStream[T](publisherUrl, subscribe, bytesToObjects, storageLevel, supervisorStrategy)
- }
-
- /**
- * Create an input stream that receives messages pushed by a zeromq publisher.
- * @param publisherUrl Url of remote zeromq publisher
- * @param subscribe topic to subscribe to
- * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
- * of byte thus it needs the converter(which might be deserializer of bytes)
- * to translate from sequence of sequence of bytes, where sequence refer to a frame
- * and sub sequence refer to its payload.
- * @param storageLevel RDD storage level. Defaults to memory-only.
- */
- def zeroMQStream[T](
- publisherUrl:String,
- subscribe: Subscribe,
- bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]],
- storageLevel: StorageLevel
- ): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
- def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
- ssc.zeroMQStream[T](publisherUrl, subscribe, fn, storageLevel)
- }
-
- /**
- * Create an input stream that receives messages pushed by a zeromq publisher.
- * @param publisherUrl Url of remote zeromq publisher
- * @param subscribe topic to subscribe to
- * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each frame has sequence
- * of byte thus it needs the converter(which might be deserializer of bytes)
- * to translate from sequence of sequence of bytes, where sequence refer to a frame
- * and sub sequence refer to its payload.
- */
- def zeroMQStream[T](
- publisherUrl:String,
- subscribe: Subscribe,
- bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]]
- ): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
- def fn(x: Seq[Seq[Byte]]) = bytesToObjects.apply(x.map(_.toArray).toArray).toIterator
- ssc.zeroMQStream[T](publisherUrl, subscribe, fn)
- }
-
- /**
* Registers an output stream that will be computed every interval
*/
def registerOutputStream(outputStream: JavaDStreamLike[_, _, _]) {
@@ -547,8 +327,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @tparam T Type of objects in the RDD
*/
def queueStream[T](queue: java.util.Queue[JavaRDD[T]]): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val sQueue = new scala.collection.mutable.Queue[RDD[T]]
sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
ssc.queueStream(sQueue)
@@ -564,8 +344,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* @tparam T Type of objects in the RDD
*/
def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val sQueue = new scala.collection.mutable.Queue[RDD[T]]
sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
ssc.queueStream(sQueue, oneAtATime)
@@ -585,8 +365,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
queue: java.util.Queue[JavaRDD[T]],
oneAtATime: Boolean,
defaultRDD: JavaRDD[T]): JavaDStream[T] = {
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val sQueue = new scala.collection.mutable.Queue[RDD[T]]
sQueue.enqueue(queue.map(_.rdd).toSeq: _*)
ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd)
@@ -597,7 +377,7 @@ class JavaStreamingContext(val ssc: StreamingContext) {
*/
def union[T](first: JavaDStream[T], rest: JList[JavaDStream[T]]): JavaDStream[T] = {
val dstreams: Seq[DStream[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream)
- implicit val cm: ClassManifest[T] = first.classManifest
+ implicit val cm: ClassTag[T] = first.classTag
ssc.union(dstreams)(cm)
}
@@ -609,9 +389,9 @@ class JavaStreamingContext(val ssc: StreamingContext) {
rest: JList[JavaPairDStream[K, V]]
): JavaPairDStream[K, V] = {
val dstreams: Seq[DStream[(K, V)]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.dstream)
- implicit val cm: ClassManifest[(K, V)] = first.classManifest
- implicit val kcm: ClassManifest[K] = first.kManifest
- implicit val vcm: ClassManifest[V] = first.vManifest
+ implicit val cm: ClassTag[(K, V)] = first.classTag
+ implicit val kcm: ClassTag[K] = first.kManifest
+ implicit val vcm: ClassTag[V] = first.vManifest
new JavaPairDStream[K, V](ssc.union(dstreams)(cm))(kcm, vcm)
}
@@ -628,8 +408,8 @@ class JavaStreamingContext(val ssc: StreamingContext) {
dstreams: JList[JavaDStream[_]],
transformFunc: JFunction2[JList[JavaRDD[_]], Time, JavaRDD[T]]
): JavaDStream[T] = {
- implicit val cmt: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cmt: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val scalaDStreams = dstreams.map(_.dstream).toSeq
val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList
@@ -651,10 +431,10 @@ class JavaStreamingContext(val ssc: StreamingContext) {
dstreams: JList[JavaDStream[_]],
transformFunc: JFunction2[JList[JavaRDD[_]], Time, JavaPairRDD[K, V]]
): JavaPairDStream[K, V] = {
- implicit val cmk: ClassManifest[K] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
- implicit val cmv: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cmk: ClassTag[K] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]]
+ implicit val cmv: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
val scalaDStreams = dstreams.map(_.dstream).toSeq
val scalaTransformFunc = (rdds: Seq[RDD[_]], time: Time) => {
val jrdds = rdds.map(rdd => JavaRDD.fromRDD[AnyRef](rdd.asInstanceOf[RDD[AnyRef]])).toList
@@ -683,6 +463,13 @@ class JavaStreamingContext(val ssc: StreamingContext) {
ssc.remember(duration)
}
+ /** Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for
+ * receiving system events related to streaming.
+ */
+ def addStreamingListener(streamingListener: StreamingListener) {
+ ssc.addStreamingListener(streamingListener)
+ }
+
/**
* Starts the execution of the streams.
*/
@@ -692,5 +479,12 @@ class JavaStreamingContext(val ssc: StreamingContext) {
* Sstops the execution of the streams.
*/
def stop() = ssc.stop()
+}
+object JavaStreamingContext {
+ /**
+ * Find the JAR from which a given class was loaded, to make it easy for users to pass
+ * their JARs to SparkContext.
+ */
+ def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
index a9a05c9981..f396c34758 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ConstantInputDStream.scala
@@ -19,11 +19,12 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Time, StreamingContext}
+import scala.reflect.ClassTag
/**
* An input stream that always returns the same RDD on each timestep. Useful for testing.
*/
-class ConstantInputDStream[T: ClassManifest](ssc_ : StreamingContext, rdd: RDD[T])
+class ConstantInputDStream[T: ClassTag](ssc_ : StreamingContext, rdd: RDD[T])
extends InputDStream[T](ssc_) {
override def start() {}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
index fea0573b77..fb9eda8996 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
@@ -17,30 +17,31 @@
package org.apache.spark.streaming.dstream
-import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.UnionRDD
-import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time}
-
+import java.io.{ObjectInputStream, IOException}
+import scala.collection.mutable.{HashSet, HashMap}
+import scala.reflect.ClassTag
import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import org.apache.spark.SparkException
+import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.streaming.{DStreamCheckpointData, StreamingContext, Time}
-import scala.collection.mutable.{HashSet, HashMap}
-import java.io.{ObjectInputStream, IOException}
private[streaming]
-class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K,V] : ClassManifest](
+class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag](
@transient ssc_ : StreamingContext,
directory: String,
filter: Path => Boolean = FileInputDStream.defaultFilter,
- newFilesOnly: Boolean = true)
+ newFilesOnly: Boolean = true)
extends InputDStream[(K, V)](ssc_) {
protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData
// Latest file mod time seen till any point of time
- private val lastModTimeFiles = new HashSet[String]()
- private var lastModTime = 0L
+ private val prevModTimeFiles = new HashSet[String]()
+ private var prevModTime = 0L
@transient private var path_ : Path = null
@transient private var fs_ : FileSystem = null
@@ -48,13 +49,13 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
override def start() {
if (newFilesOnly) {
- lastModTime = graph.zeroTime.milliseconds
+ prevModTime = graph.zeroTime.milliseconds
} else {
- lastModTime = 0
+ prevModTime = 0
}
- logDebug("LastModTime initialized to " + lastModTime + ", new files only = " + newFilesOnly)
+ logDebug("LastModTime initialized to " + prevModTime + ", new files only = " + newFilesOnly)
}
-
+
override def stop() { }
/**
@@ -67,55 +68,22 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
* the previous call.
*/
override def compute(validTime: Time): Option[RDD[(K, V)]] = {
- assert(validTime.milliseconds >= lastModTime, "Trying to get new files for really old time [" + validTime + " < " + lastModTime)
-
- // Create the filter for selecting new files
- val newFilter = new PathFilter() {
- // Latest file mod time seen in this round of fetching files and its corresponding files
- var latestModTime = 0L
- val latestModTimeFiles = new HashSet[String]()
+ assert(validTime.milliseconds >= prevModTime,
+ "Trying to get new files for really old time [" + validTime + " < " + prevModTime + "]")
- def accept(path: Path): Boolean = {
- if (!filter(path)) { // Reject file if it does not satisfy filter
- logDebug("Rejected by filter " + path)
- return false
- } else { // Accept file only if
- val modTime = fs.getFileStatus(path).getModificationTime()
- logDebug("Mod time for " + path + " is " + modTime)
- if (modTime < lastModTime) {
- logDebug("Mod time less than last mod time")
- return false // If the file was created before the last time it was called
- } else if (modTime == lastModTime && lastModTimeFiles.contains(path.toString)) {
- logDebug("Mod time equal to last mod time, but file considered already")
- return false // If the file was created exactly as lastModTime but not reported yet
- } else if (modTime > validTime.milliseconds) {
- logDebug("Mod time more than valid time")
- return false // If the file was created after the time this function call requires
- }
- if (modTime > latestModTime) {
- latestModTime = modTime
- latestModTimeFiles.clear()
- logDebug("Latest mod time updated to " + latestModTime)
- }
- latestModTimeFiles += path.toString
- logDebug("Accepted " + path)
- return true
- }
- }
- }
- logDebug("Finding new files at time " + validTime + " for last mod time = " + lastModTime)
- val newFiles = fs.listStatus(path, newFilter).map(_.getPath.toString)
+ // Find new files
+ val (newFiles, latestModTime, latestModTimeFiles) = findNewFiles(validTime.milliseconds)
logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n"))
if (newFiles.length > 0) {
// Update the modification time and the files processed for that modification time
- if (lastModTime != newFilter.latestModTime) {
- lastModTime = newFilter.latestModTime
- lastModTimeFiles.clear()
+ if (prevModTime < latestModTime) {
+ prevModTime = latestModTime
+ prevModTimeFiles.clear()
}
- lastModTimeFiles ++= newFilter.latestModTimeFiles
- logDebug("Last mod time updated to " + lastModTime)
+ prevModTimeFiles ++= latestModTimeFiles
+ logDebug("Last mod time updated to " + prevModTime)
}
- files += ((validTime, newFiles))
+ files += ((validTime, newFiles.toArray))
Some(filesToRDD(newFiles))
}
@@ -130,12 +98,28 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n"))
}
+ /**
+ * Find files which have modification timestamp <= current time and return a 3-tuple of
+ * (new files found, latest modification time among them, files with latest modification time)
+ */
+ private def findNewFiles(currentTime: Long): (Seq[String], Long, Seq[String]) = {
+ logDebug("Trying to get new files for time " + currentTime)
+ val filter = new CustomPathFilter(currentTime)
+ val newFiles = fs.listStatus(path, filter).map(_.getPath.toString)
+ (newFiles, filter.latestModTime, filter.latestModTimeFiles.toSeq)
+ }
+
/** Generate one RDD from an array of files */
- protected[streaming] def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
- new UnionRDD(
- context.sparkContext,
- files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file))
- )
+ private def filesToRDD(files: Seq[String]): RDD[(K, V)] = {
+ val fileRDDs = files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file))
+ files.zip(fileRDDs).foreach { case (file, rdd) => {
+ if (rdd.partitions.size == 0) {
+ logError("File " + file + " has no data in it. Spark Streaming can only ingest " +
+ "files that have been \"moved\" to the directory assigned to the file stream. " +
+ "Refer to the streaming programming guide for more details.")
+ }
+ }}
+ new UnionRDD(context.sparkContext, fileRDDs)
}
private def path: Path = {
@@ -148,6 +132,10 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
fs_
}
+ private def reset() {
+ fs_ = null
+ }
+
@throws(classOf[IOException])
private def readObject(ois: ObjectInputStream) {
logDebug(this.getClass().getSimpleName + ".readObject used")
@@ -189,11 +177,54 @@ class FileInputDStream[K: ClassManifest, V: ClassManifest, F <: NewInputFormat[K
hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]"
}
}
+
+ /**
+ * Custom PathFilter class to find new files that have modification timestamps <= current time,
+ * but have not been seen before (i.e. the file should not be in lastModTimeFiles)
+ */
+ private[streaming]
+ class CustomPathFilter(maxModTime: Long) extends PathFilter {
+ // Latest file mod time seen in this round of fetching files and its corresponding files
+ var latestModTime = 0L
+ val latestModTimeFiles = new HashSet[String]()
+
+ def accept(path: Path): Boolean = {
+ try {
+ if (!filter(path)) { // Reject file if it does not satisfy filter
+ logDebug("Rejected by filter " + path)
+ return false
+ }
+ val modTime = fs.getFileStatus(path).getModificationTime()
+ logDebug("Mod time for " + path + " is " + modTime)
+ if (modTime < prevModTime) {
+ logDebug("Mod time less than last mod time")
+ return false // If the file was created before the last time it was called
+ } else if (modTime == prevModTime && prevModTimeFiles.contains(path.toString)) {
+ logDebug("Mod time equal to last mod time, but file considered already")
+ return false // If the file was created exactly as lastModTime but not reported yet
+ } else if (modTime > maxModTime) {
+ logDebug("Mod time more than ")
+ return false // If the file is too new that considering it may give errors
+ }
+ if (modTime > latestModTime) {
+ latestModTime = modTime
+ latestModTimeFiles.clear()
+ logDebug("Latest mod time updated to " + latestModTime)
+ }
+ latestModTimeFiles += path.toString
+ logDebug("Accepted " + path)
+ } catch {
+ case fnfe: java.io.FileNotFoundException =>
+ logWarning("Error finding new files", fnfe)
+ reset()
+ return false
+ }
+ return true
+ }
+ }
}
private[streaming]
object FileInputDStream {
def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".")
}
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
index 91ee2c1a36..db2e0a4cee 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FilteredDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class FilteredDStream[T: ClassManifest](
+class FilteredDStream[T: ClassTag](
parent: DStream[T],
filterFunc: T => Boolean
) extends DStream[T](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
index ca7d7ca49e..244dc3ee4f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMapValuedDStream.scala
@@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
+import scala.reflect.ClassTag
private[streaming]
-class FlatMapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+class FlatMapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag](
parent: DStream[(K, V)],
flatMapValueFunc: V => TraversableOnce[U]
) extends DStream[(K, U)](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
index b37966f9a7..336c4b7a92 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FlatMappedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class FlatMappedDStream[T: ClassManifest, U: ClassManifest](
+class FlatMappedDStream[T: ClassTag, U: ClassTag](
parent: DStream[T],
flatMapFunc: T => Traversable[U]
) extends DStream[U](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
index e21bac4602..364abcde68 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala
@@ -18,10 +18,12 @@
package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
-import org.apache.spark.streaming.{Duration, DStream, Job, Time}
+import org.apache.spark.streaming.{Duration, DStream, Time}
+import org.apache.spark.streaming.scheduler.Job
+import scala.reflect.ClassTag
private[streaming]
-class ForEachDStream[T: ClassManifest] (
+class ForEachDStream[T: ClassTag] (
parent: DStream[T],
foreachFunc: (RDD[T], Time) => Unit
) extends DStream[Unit](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
index 4294b07d91..23136f44fa 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/GlommedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class GlommedDStream[T: ClassManifest](parent: DStream[T])
+class GlommedDStream[T: ClassTag](parent: DStream[T])
extends DStream[Array[T]](parent.ssc) {
override def dependencies = List(parent)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
index 674b27118c..f01e67fe13 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
@@ -19,6 +19,8 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Time, Duration, StreamingContext, DStream}
+import scala.reflect.ClassTag
+
/**
* This is the abstract base class for all input streams. This class provides to methods
* start() and stop() which called by the scheduler to start and stop receiving data/
@@ -30,7 +32,7 @@ import org.apache.spark.streaming.{Time, Duration, StreamingContext, DStream}
* that requires running a receiver on the worker nodes, use NetworkInputDStream
* as the parent class.
*/
-abstract class InputDStream[T: ClassManifest] (@transient ssc_ : StreamingContext)
+abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext)
extends DStream[T](ssc_) {
var lastValidTime: Time = null
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
index 5329601a6f..8a04060e5b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapPartitionedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class MapPartitionedDStream[T: ClassManifest, U: ClassManifest](
+class MapPartitionedDStream[T: ClassTag, U: ClassTag](
parent: DStream[T],
mapPartFunc: Iterator[T] => Iterator[U],
preservePartitioning: Boolean
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
index 8290df90a2..0ce364fd46 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MapValuedDStream.scala
@@ -20,9 +20,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
+import scala.reflect.ClassTag
private[streaming]
-class MapValuedDStream[K: ClassManifest, V: ClassManifest, U: ClassManifest](
+class MapValuedDStream[K: ClassTag, V: ClassTag, U: ClassTag](
parent: DStream[(K, V)],
mapValueFunc: V => U
) extends DStream[(K, U)](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
index b1682afea3..c0b7491d09 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/MappedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.{Duration, DStream, Time}
import org.apache.spark.rdd.RDD
+import scala.reflect.ClassTag
private[streaming]
-class MappedDStream[T: ClassManifest, U: ClassManifest] (
+class MappedDStream[T: ClassTag, U: ClassTag] (
parent: DStream[T],
mapFunc: T => U
) extends DStream[U](parent.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
index a82862c802..27d474c0a0 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
@@ -21,17 +21,19 @@ import java.util.concurrent.ArrayBlockingQueue
import java.nio.ByteBuffer
import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.Await
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
import akka.actor.{Props, Actor}
import akka.pattern.ask
-import akka.dispatch.Await
-import akka.util.duration._
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.{BlockId, StorageLevel, StreamBlockId}
+import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver}
/**
* Abstract class for defining any InputDStream that has to start a receiver on worker
@@ -42,7 +44,7 @@ import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId}
* @param ssc_ Streaming context that will execute this input stream
* @tparam T Class type of the object of this stream
*/
-abstract class NetworkInputDStream[T: ClassManifest](@transient ssc_ : StreamingContext)
+abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext)
extends InputDStream[T](ssc_) {
// This is an unique identifier that is used to match the network receiver with the
@@ -84,9 +86,7 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe
* Abstract class of a receiver that can be run on worker nodes to receive external data. See
* [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation.
*/
-abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Logging {
-
- initLogging()
+abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging {
lazy protected val env = SparkEnv.get
@@ -174,10 +174,10 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
/** A helper actor that communicates with the NetworkInputTracker */
private class NetworkReceiverActor extends Actor {
logInfo("Attempting to register with tracker")
- val ip = System.getProperty("spark.driver.host", "localhost")
- val port = System.getProperty("spark.driver.port", "7077").toInt
- val url = "akka://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
- val tracker = env.actorSystem.actorFor(url)
+ val ip = env.conf.get("spark.driver.host", "localhost")
+ val port = env.conf.get("spark.driver.port", "7077").toInt
+ val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port)
+ val tracker = env.actorSystem.actorSelection(url)
val timeout = 5.seconds
override def preStart() {
@@ -212,7 +212,7 @@ abstract class NetworkReceiver[T: ClassManifest]() extends Serializable with Log
case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null)
val clock = new SystemClock()
- val blockInterval = System.getProperty("spark.streaming.blockInterval", "200").toLong
+ val blockInterval = env.conf.get("spark.streaming.blockInterval", "200").toLong
val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer)
val blockStorageLevel = storageLevel
val blocksForPushing = new ArrayBlockingQueue[Block](1000)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
index 15782f5c11..6f9477020a 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala
@@ -18,9 +18,10 @@
package org.apache.spark.streaming.dstream
import org.apache.spark.streaming.StreamingContext
+import scala.reflect.ClassTag
private[streaming]
-class PluggableInputDStream[T: ClassManifest](
+class PluggableInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
index 7d9f3521b1..97325f8ea3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala
@@ -19,13 +19,13 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.UnionRDD
-
import scala.collection.mutable.Queue
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.streaming.{Time, StreamingContext}
+import scala.reflect.ClassTag
private[streaming]
-class QueueInputDStream[T: ClassManifest](
+class QueueInputDStream[T: ClassTag](
@transient ssc: StreamingContext,
val queue: Queue[RDD[T]],
oneAtATime: Boolean,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala
index 10ed4ef78d..dea0f26f90 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
@@ -21,6 +21,8 @@ import org.apache.spark.Logging
import org.apache.spark.storage.{StorageLevel, StreamBlockId}
import org.apache.spark.streaming.StreamingContext
+import scala.reflect.ClassTag
+
import java.net.InetSocketAddress
import java.nio.ByteBuffer
import java.nio.channels.{ReadableByteChannel, SocketChannel}
@@ -35,7 +37,7 @@ import java.util.concurrent.ArrayBlockingQueue
* in the format that the system is configured with.
*/
private[streaming]
-class RawInputDStream[T: ClassManifest](
+class RawInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
host: String,
port: Int,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
index b88a4db959..db56345ca8 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -28,8 +28,11 @@ import org.apache.spark.storage.StorageLevel
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
private[streaming]
-class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
+class ReducedWindowedDStream[K: ClassTag, V: ClassTag](
parent: DStream[(K, V)],
reduceFunc: (V, V) => V,
invReduceFunc: (V, V) => V,
@@ -49,7 +52,7 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
"must be multiple of the slide duration of parent DStream (" + parent.slideDuration + ")"
)
- // Reduce each batch of data using reduceByKey which will be further reduced by window
+ // Reduce each batch of data using reduceByKey which will be further reduced by window
// by ReducedWindowedDStream
val reducedStream = parent.reduceByKey(reduceFunc, partitioner)
@@ -170,5 +173,3 @@ class ReducedWindowedDStream[K: ClassManifest, V: ClassManifest](
}
}
}
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
index a95e66d761..84e69f277b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ShuffledDStream.scala
@@ -21,15 +21,17 @@ import org.apache.spark.Partitioner
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
import org.apache.spark.streaming.{Duration, DStream, Time}
+import scala.reflect.ClassTag
private[streaming]
-class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
+class ShuffledDStream[K: ClassTag, V: ClassTag, C: ClassTag](
parent: DStream[(K,V)],
createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiner: (C, C) => C,
- partitioner: Partitioner
- ) extends DStream [(K,C)] (parent.ssc) {
+ partitioner: Partitioner,
+ mapSideCombine: Boolean = true
+ ) extends DStream[(K,C)] (parent.ssc) {
override def dependencies = List(parent)
@@ -37,8 +39,8 @@ class ShuffledDStream[K: ClassManifest, V: ClassManifest, C: ClassManifest](
override def compute(validTime: Time): Option[RDD[(K,C)]] = {
parent.getOrCompute(validTime) match {
- case Some(rdd) =>
- Some(rdd.combineByKey[C](createCombiner, mergeValue, mergeCombiner, partitioner))
+ case Some(rdd) => Some(rdd.combineByKey[C](
+ createCombiner, mergeValue, mergeCombiner, partitioner, mapSideCombine))
case None => None
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
index e2539c7396..2cdd13f205 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala
@@ -21,11 +21,13 @@ import org.apache.spark.streaming.StreamingContext
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.NextIterator
+import scala.reflect.ClassTag
+
import java.io._
import java.net.Socket
private[streaming]
-class SocketInputDStream[T: ClassManifest](
+class SocketInputDStream[T: ClassTag](
@transient ssc_ : StreamingContext,
host: String,
port: Int,
@@ -39,7 +41,7 @@ class SocketInputDStream[T: ClassManifest](
}
private[streaming]
-class SocketReceiver[T: ClassManifest](
+class SocketReceiver[T: ClassTag](
host: String,
port: Int,
bytesToObjects: InputStream => Iterator[T],
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
index 362a6bf4cc..e0ff3ccba4 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala
@@ -23,8 +23,10 @@ import org.apache.spark.SparkContext._
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.{Duration, Time, DStream}
+import scala.reflect.ClassTag
+
private[streaming]
-class StateDStream[K: ClassManifest, V: ClassManifest, S: ClassManifest](
+class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag](
parent: DStream[(K, V)],
updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)],
partitioner: Partitioner,
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
index 71bcb2b390..aeea060df7 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala
@@ -19,9 +19,10 @@ package org.apache.spark.streaming.dstream
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Duration, DStream, Time}
+import scala.reflect.ClassTag
private[streaming]
-class TransformedDStream[U: ClassManifest] (
+class TransformedDStream[U: ClassTag] (
parents: Seq[DStream[_]],
transformFunc: (Seq[RDD[_]], Time) => RDD[U]
) extends DStream[U](parents.head.ssc) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
index c696bb70a8..0d84ec84f2 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
@@ -22,8 +22,11 @@ import org.apache.spark.rdd.RDD
import collection.mutable.ArrayBuffer
import org.apache.spark.rdd.UnionRDD
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
+
private[streaming]
-class UnionDStream[T: ClassManifest](parents: Array[DStream[T]])
+class UnionDStream[T: ClassTag](parents: Array[DStream[T]])
extends DStream[T](parents.head.ssc) {
if (parents.length == 0) {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
index 3c57294269..89c43ff935 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
@@ -17,13 +17,15 @@
package org.apache.spark.streaming.dstream
-import org.apache.spark.rdd.RDD
-import org.apache.spark.rdd.UnionRDD
+import org.apache.spark.rdd.{PartitionerAwareUnionRDD, RDD, UnionRDD}
import org.apache.spark.storage.StorageLevel
-import org.apache.spark.streaming.{Duration, Interval, Time, DStream}
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.Duration
+
+import scala.reflect.ClassTag
private[streaming]
-class WindowedDStream[T: ClassManifest](
+class WindowedDStream[T: ClassTag](
parent: DStream[T],
_windowDuration: Duration,
_slideDuration: Duration)
@@ -49,9 +51,14 @@ class WindowedDStream[T: ClassManifest](
override def compute(validTime: Time): Option[RDD[T]] = {
val currentWindow = new Interval(validTime - windowDuration + parent.slideDuration, validTime)
- Some(new UnionRDD(ssc.sc, parent.slice(currentWindow)))
+ val rddsInWindow = parent.slice(currentWindow)
+ val windowRDD = if (rddsInWindow.flatMap(_.partitioner).distinct.length == 1) {
+ logDebug("Using partition aware union for windowing at " + validTime)
+ new PartitionerAwareUnionRDD(ssc.sc, rddsInWindow)
+ } else {
+ logDebug("Using normal union for windowing at " + validTime)
+ new UnionRDD(ssc.sc,rddsInWindow)
+ }
+ Some(windowRDD)
}
}
-
-
-
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 ef0f85a717..fdf5371a89 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
@@ -20,6 +20,10 @@ package org.apache.spark.streaming.receivers
import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy }
import akka.actor.{ actorRef2Scala, ActorRef }
import akka.actor.{ PossiblyHarmful, OneForOneStrategy }
+import akka.actor.SupervisorStrategy._
+
+import scala.concurrent.duration._
+import scala.reflect.ClassTag
import org.apache.spark.storage.{StorageLevel, StreamBlockId}
import org.apache.spark.streaming.dstream.NetworkReceiver
@@ -28,12 +32,9 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.collection.mutable.ArrayBuffer
-/** A helper with set of defaults for supervisor strategy **/
+/** A helper with set of defaults for supervisor strategy */
object ReceiverSupervisorStrategy {
- import akka.util.duration._
- import akka.actor.SupervisorStrategy._
-
val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange =
15 millis) {
case _: RuntimeException ⇒ Restart
@@ -48,10 +49,10 @@ object ReceiverSupervisorStrategy {
* Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html
*
* @example {{{
- * class MyActor extends Actor with Receiver{
- * def receive {
- * case anything :String ⇒ pushBlock(anything)
- * }
+ * class MyActor extends Actor with Receiver{
+ * def receive {
+ * case anything :String => pushBlock(anything)
+ * }
* }
* //Can be plugged in actorStream as follows
* ssc.actorStream[String](Props(new MyActor),"MyActorReceiver")
@@ -65,11 +66,11 @@ object ReceiverSupervisorStrategy {
*
*/
trait Receiver { self: Actor ⇒
- def pushBlock[T: ClassManifest](iter: Iterator[T]) {
+ def pushBlock[T: ClassTag](iter: Iterator[T]) {
context.parent ! Data(iter)
}
- def pushBlock[T: ClassManifest](data: T) {
+ def pushBlock[T: ClassTag](data: T) {
context.parent ! Data(data)
}
@@ -83,8 +84,8 @@ case class Statistics(numberOfMsgs: Int,
numberOfHiccups: Int,
otherInfo: String)
-/** Case class to receive data sent by child actors **/
-private[streaming] case class Data[T: ClassManifest](data: T)
+/** Case class to receive data sent by child actors */
+private[streaming] case class Data[T: ClassTag](data: T)
/**
* Provides Actors as receivers for receiving stream.
@@ -95,19 +96,19 @@ private[streaming] case class Data[T: ClassManifest](data: T)
* his own Actor to run as receiver for Spark Streaming input source.
*
* This starts a supervisor actor which starts workers and also provides
- * [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance].
- *
+ * [http://doc.akka.io/docs/akka/2.0.5/scala/fault-tolerance.html fault-tolerance].
+ *
* Here's a way to start more supervisor/workers as its children.
*
* @example {{{
- * context.parent ! Props(new Supervisor)
+ * context.parent ! Props(new Supervisor)
* }}} OR {{{
* context.parent ! Props(new Worker,"Worker")
* }}}
*
*
*/
-private[streaming] class ActorReceiver[T: ClassManifest](
+private[streaming] class ActorReceiver[T: ClassTag](
props: Props,
name: String,
storageLevel: StorageLevel,
@@ -120,7 +121,7 @@ private[streaming] class ActorReceiver[T: ClassManifest](
protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor),
"Supervisor" + streamId)
- private class Supervisor extends Actor {
+ class Supervisor extends Actor {
override val supervisorStrategy = receiverSupervisorStrategy
val worker = context.actorOf(props, name)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
new file mode 100644
index 0000000000..4e8d07fe92
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.scheduler
+
+import org.apache.spark.streaming.Time
+
+/**
+ * Class having information on completed batches.
+ * @param batchTime Time of the batch
+ * @param submissionTime Clock time of when jobs of this batch was submitted to
+ * the streaming scheduler queue
+ * @param processingStartTime Clock time of when the first job of this batch started processing
+ * @param processingEndTime Clock time of when the last job of this batch finished processing
+ */
+case class BatchInfo(
+ batchTime: Time,
+ submissionTime: Long,
+ processingStartTime: Option[Long],
+ processingEndTime: Option[Long]
+ ) {
+
+ /**
+ * Time taken for the first job of this batch to start processing from the time this batch
+ * was submitted to the streaming scheduler. Essentially, it is
+ * `processingStartTime` - `submissionTime`.
+ */
+ def schedulingDelay = processingStartTime.map(_ - submissionTime)
+
+ /**
+ * Time taken for the all jobs of this batch to finish processing from the time they started
+ * processing. Essentially, it is `processingEndTime` - `processingStartTime`.
+ */
+ def processingDelay = processingEndTime.zip(processingStartTime).map(x => x._1 - x._2).headOption
+
+ /**
+ * Time taken for all the jobs of this batch to finish processing from the time they
+ * were submitted. Essentially, it is `processingDelay` + `schedulingDelay`.
+ */
+ def totalDelay = schedulingDelay.zip(processingDelay).map(x => x._1 + x._2).headOption
+}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Job.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala
index 2128b7c7a6..7341bfbc99 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Job.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/Job.scala
@@ -15,13 +15,17 @@
* limitations under the License.
*/
-package org.apache.spark.streaming
+package org.apache.spark.streaming.scheduler
-import java.util.concurrent.atomic.AtomicLong
+import org.apache.spark.streaming.Time
+/**
+ * Class representing a Spark computation. It may contain multiple Spark jobs.
+ */
private[streaming]
class Job(val time: Time, func: () => _) {
- val id = Job.getNewId()
+ var id: String = _
+
def run(): Long = {
val startTime = System.currentTimeMillis
func()
@@ -29,13 +33,9 @@ class Job(val time: Time, func: () => _) {
(stopTime - startTime)
}
- override def toString = "streaming job " + id + " @ " + time
-}
-
-private[streaming]
-object Job {
- val id = new AtomicLong(0)
-
- def getNewId() = id.getAndIncrement()
-}
+ def setId(number: Int) {
+ id = "streaming job " + time + "." + number
+ }
+ override def toString = id
+} \ No newline at end of file
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index ed892e33e6..5f8be93a98 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Scheduler.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -15,65 +15,96 @@
* limitations under the License.
*/
-package org.apache.spark.streaming
+package org.apache.spark.streaming.scheduler
-import util.{ManualClock, RecurringTimer, Clock}
+import akka.actor.{Props, Actor}
import org.apache.spark.SparkEnv
import org.apache.spark.Logging
-
+import org.apache.spark.streaming.{Checkpoint, Time, CheckpointWriter}
+import org.apache.spark.streaming.util.{ManualClock, RecurringTimer, Clock}
+
+/** Event classes for JobGenerator */
+private[scheduler] sealed trait JobGeneratorEvent
+private[scheduler] case class GenerateJobs(time: Time) extends JobGeneratorEvent
+private[scheduler] case class ClearOldMetadata(time: Time) extends JobGeneratorEvent
+private[scheduler] case class DoCheckpoint(time: Time) extends JobGeneratorEvent
+
+/**
+ * This class generates jobs from DStreams as well as drives checkpointing and cleaning
+ * up DStream metadata.
+ */
private[streaming]
-class Scheduler(ssc: StreamingContext) extends Logging {
-
- initLogging()
+class JobGenerator(jobScheduler: JobScheduler) extends Logging {
- val concurrentJobs = System.getProperty("spark.streaming.concurrentJobs", "1").toInt
- val jobManager = new JobManager(ssc, concurrentJobs)
- val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
- new CheckpointWriter(ssc.checkpointDir)
+ val ssc = jobScheduler.ssc
+ val graph = ssc.graph
+ val eventProcessorActor = ssc.env.actorSystem.actorOf(Props(new Actor {
+ def receive = {
+ case event: JobGeneratorEvent =>
+ logDebug("Got event of type " + event.getClass.getName)
+ processEvent(event)
+ }
+ }))
+ val clock = {
+ val clockClass = ssc.sc.conf.get(
+ "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
+ Class.forName(clockClass).newInstance().asInstanceOf[Clock]
+ }
+ val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
+ longTime => eventProcessorActor ! GenerateJobs(new Time(longTime)))
+ lazy val checkpointWriter = if (ssc.checkpointDuration != null && ssc.checkpointDir != null) {
+ new CheckpointWriter(ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration)
} else {
null
}
- val clockClass = System.getProperty(
- "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
- val clock = Class.forName(clockClass).newInstance().asInstanceOf[Clock]
- val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds,
- longTime => generateJobs(new Time(longTime)))
- val graph = ssc.graph
- var latestTime: Time = null
-
def start() = synchronized {
if (ssc.isCheckpointPresent) {
restart()
} else {
startFirstTime()
}
- logInfo("Scheduler started")
}
-
- def stop() = synchronized {
+
+ def stop() {
timer.stop()
- jobManager.stop()
if (checkpointWriter != null) checkpointWriter.stop()
ssc.graph.stop()
- logInfo("Scheduler stopped")
+ logInfo("JobGenerator stopped")
}
+ /**
+ * On batch completion, clear old metadata and checkpoint computation.
+ */
+ private[scheduler] def onBatchCompletion(time: Time) {
+ eventProcessorActor ! ClearOldMetadata(time)
+ }
+
+ /** Processes all events */
+ private def processEvent(event: JobGeneratorEvent) {
+ event match {
+ case GenerateJobs(time) => generateJobs(time)
+ case ClearOldMetadata(time) => clearOldMetadata(time)
+ case DoCheckpoint(time) => doCheckpoint(time)
+ }
+ }
+
+ /** Starts the generator for the first time */
private def startFirstTime() {
val startTime = new Time(timer.getStartTime())
graph.start(startTime - graph.batchDuration)
timer.start(startTime.milliseconds)
- logInfo("Scheduler's timer started at " + startTime)
+ logInfo("JobGenerator started at " + startTime)
}
+ /** Restarts the generator based on the information in checkpoint */
private def restart() {
-
// If manual clock is being used for testing, then
// either set the manual clock to the last checkpointed time,
// or if the property is defined set it to that time
if (clock.isInstanceOf[ManualClock]) {
val lastTime = ssc.initialCheckpoint.checkpointTime.milliseconds
- val jumpTime = System.getProperty("spark.streaming.manualClock.jump", "0").toLong
+ val jumpTime = ssc.sc.conf.get("spark.streaming.manualClock.jump", "0").toLong
clock.asInstanceOf[ManualClock].setTime(lastTime + jumpTime)
}
@@ -93,35 +124,31 @@ class Scheduler(ssc: StreamingContext) extends Logging {
val timesToReschedule = (pendingTimes ++ downTimes).distinct.sorted(Time.ordering)
logInfo("Batches to reschedule: " + timesToReschedule.mkString(", "))
timesToReschedule.foreach(time =>
- graph.generateJobs(time).foreach(jobManager.runJob)
+ jobScheduler.runJobs(time, graph.generateJobs(time))
)
// Restart the timer
timer.start(restartTime.milliseconds)
- logInfo("Scheduler's timer restarted at " + restartTime)
+ logInfo("JobGenerator restarted at " + restartTime)
}
/** Generate jobs and perform checkpoint for the given `time`. */
- def generateJobs(time: Time) {
+ private def generateJobs(time: Time) {
SparkEnv.set(ssc.env)
logInfo("\n-----------------------------------------------------\n")
- graph.generateJobs(time).foreach(jobManager.runJob)
- latestTime = time
- doCheckpoint(time)
+ jobScheduler.runJobs(time, graph.generateJobs(time))
+ eventProcessorActor ! DoCheckpoint(time)
}
- /**
- * Clear old metadata assuming jobs of `time` have finished processing.
- * And also perform checkpoint.
- */
- def clearOldMetadata(time: Time) {
+ /** Clear DStream metadata for the given `time`. */
+ private def clearOldMetadata(time: Time) {
ssc.graph.clearOldMetadata(time)
- doCheckpoint(time)
+ eventProcessorActor ! DoCheckpoint(time)
}
/** Perform checkpoint for the give `time`. */
- def doCheckpoint(time: Time) = synchronized {
- if (ssc.checkpointDuration != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) {
+ private def doCheckpoint(time: Time) = synchronized {
+ if (checkpointWriter != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) {
logInfo("Checkpointing graph for time " + time)
ssc.graph.updateCheckpointData(time)
checkpointWriter.write(new Checkpoint(ssc, time))
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
new file mode 100644
index 0000000000..9304fc1a93
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.scheduler
+
+import org.apache.spark.Logging
+import org.apache.spark.SparkEnv
+import java.util.concurrent.{TimeUnit, ConcurrentHashMap, Executors}
+import scala.collection.mutable.HashSet
+import org.apache.spark.streaming._
+
+/**
+ * This class schedules jobs to be run on Spark. It uses the JobGenerator to generate
+ * the jobs and runs them using a thread pool. Number of threads
+ */
+private[streaming]
+class JobScheduler(val ssc: StreamingContext) extends Logging {
+
+ val jobSets = new ConcurrentHashMap[Time, JobSet]
+ val numConcurrentJobs = ssc.conf.get("spark.streaming.concurrentJobs", "1").toInt
+ val executor = Executors.newFixedThreadPool(numConcurrentJobs)
+ val generator = new JobGenerator(this)
+ val listenerBus = new StreamingListenerBus()
+
+ def clock = generator.clock
+
+ def start() {
+ generator.start()
+ }
+
+ def stop() {
+ generator.stop()
+ executor.shutdown()
+ if (!executor.awaitTermination(5, TimeUnit.SECONDS)) {
+ executor.shutdownNow()
+ }
+ }
+
+ def runJobs(time: Time, jobs: Seq[Job]) {
+ if (jobs.isEmpty) {
+ logInfo("No jobs added for time " + time)
+ } else {
+ val jobSet = new JobSet(time, jobs)
+ jobSets.put(time, jobSet)
+ jobSet.jobs.foreach(job => executor.execute(new JobHandler(job)))
+ logInfo("Added jobs for time " + time)
+ }
+ }
+
+ def getPendingTimes(): Array[Time] = {
+ jobSets.keySet.toArray(new Array[Time](0))
+ }
+
+ private def beforeJobStart(job: Job) {
+ val jobSet = jobSets.get(job.time)
+ if (!jobSet.hasStarted) {
+ listenerBus.post(StreamingListenerBatchStarted(jobSet.toBatchInfo()))
+ }
+ jobSet.beforeJobStart(job)
+ logInfo("Starting job " + job.id + " from job set of time " + jobSet.time)
+ SparkEnv.set(generator.ssc.env)
+ }
+
+ private def afterJobEnd(job: Job) {
+ val jobSet = jobSets.get(job.time)
+ jobSet.afterJobStop(job)
+ logInfo("Finished job " + job.id + " from job set of time " + jobSet.time)
+ if (jobSet.hasCompleted) {
+ jobSets.remove(jobSet.time)
+ generator.onBatchCompletion(jobSet.time)
+ logInfo("Total delay: %.3f s for time %s (execution: %.3f s)".format(
+ jobSet.totalDelay / 1000.0, jobSet.time.toString,
+ jobSet.processingDelay / 1000.0
+ ))
+ listenerBus.post(StreamingListenerBatchCompleted(jobSet.toBatchInfo()))
+ }
+ }
+
+ private[streaming]
+ class JobHandler(job: Job) extends Runnable {
+ def run() {
+ beforeJobStart(job)
+ try {
+ job.run()
+ } catch {
+ case e: Exception =>
+ logError("Running " + job + " failed", e)
+ }
+ afterJobEnd(job)
+ }
+ }
+}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala
new file mode 100644
index 0000000000..57268674ea
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.scheduler
+
+import scala.collection.mutable.HashSet
+import org.apache.spark.streaming.Time
+
+/** Class representing a set of Jobs
+ * belong to the same batch.
+ */
+private[streaming]
+case class JobSet(time: Time, jobs: Seq[Job]) {
+
+ private val incompleteJobs = new HashSet[Job]()
+ var submissionTime = System.currentTimeMillis() // when this jobset was submitted
+ var processingStartTime = -1L // when the first job of this jobset started processing
+ var processingEndTime = -1L // when the last job of this jobset finished processing
+
+ jobs.zipWithIndex.foreach { case (job, i) => job.setId(i) }
+ incompleteJobs ++= jobs
+
+ def beforeJobStart(job: Job) {
+ if (processingStartTime < 0) processingStartTime = System.currentTimeMillis()
+ }
+
+ def afterJobStop(job: Job) {
+ incompleteJobs -= job
+ if (hasCompleted) processingEndTime = System.currentTimeMillis()
+ }
+
+ def hasStarted() = (processingStartTime > 0)
+
+ def hasCompleted() = incompleteJobs.isEmpty
+
+ // Time taken to process all the jobs from the time they started processing
+ // (i.e. not including the time they wait in the streaming scheduler queue)
+ def processingDelay = processingEndTime - processingStartTime
+
+ // Time taken to process all the jobs from the time they were submitted
+ // (i.e. including the time they wait in the streaming scheduler queue)
+ def totalDelay = {
+ processingEndTime - time.milliseconds
+ }
+
+ def toBatchInfo(): BatchInfo = {
+ new BatchInfo(
+ time,
+ submissionTime,
+ if (processingStartTime >= 0 ) Some(processingStartTime) else None,
+ if (processingEndTime >= 0 ) Some(processingEndTime) else None
+ )
+ }
+}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
index b97fb7e6e3..75f7244643 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/NetworkInputTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming
+package org.apache.spark.streaming.scheduler
import org.apache.spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver}
import org.apache.spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError}
@@ -25,12 +25,13 @@ import org.apache.spark.SparkContext._
import scala.collection.mutable.HashMap
import scala.collection.mutable.Queue
+import scala.concurrent.duration._
import akka.actor._
import akka.pattern.ask
-import akka.util.duration._
import akka.dispatch._
import org.apache.spark.storage.BlockId
+import org.apache.spark.streaming.{Time, StreamingContext}
private[streaming] sealed trait NetworkInputTrackerMessage
private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) extends NetworkInputTrackerMessage
@@ -159,7 +160,9 @@ class NetworkInputTracker(
}
// Run the dummy Spark job to ensure that all slaves have registered.
// This avoids all the receivers to be scheduled on the same node.
- ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
+ if (!ssc.sparkContext.isLocal) {
+ ssc.sparkContext.makeRDD(1 to 50, 50).map(x => (x, 1)).reduceByKey(_ + _, 20).collect()
+ }
// Distribute the receivers and start them
ssc.sparkContext.runJob(tempRDD, startReceiver)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala
new file mode 100644
index 0000000000..36225e190c
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.scheduler
+
+import scala.collection.mutable.Queue
+import org.apache.spark.util.Distribution
+
+/** Base trait for events related to StreamingListener */
+sealed trait StreamingListenerEvent
+
+case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent
+
+case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent
+
+
+/**
+ * A listener interface for receiving information about an ongoing streaming
+ * computation.
+ */
+trait StreamingListener {
+ /**
+ * Called when processing of a batch has completed
+ */
+ def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { }
+
+ /**
+ * Called when processing of a batch has started
+ */
+ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { }
+}
+
+
+/**
+ * A simple StreamingListener that logs summary statistics across Spark Streaming batches
+ * @param numBatchInfos Number of last batches to consider for generating statistics (default: 10)
+ */
+class StatsReportListener(numBatchInfos: Int = 10) extends StreamingListener {
+ // Queue containing latest completed batches
+ val batchInfos = new Queue[BatchInfo]()
+
+ override def onBatchCompleted(batchStarted: StreamingListenerBatchCompleted) {
+ batchInfos.enqueue(batchStarted.batchInfo)
+ if (batchInfos.size > numBatchInfos) batchInfos.dequeue()
+ printStats()
+ }
+
+ def printStats() {
+ showMillisDistribution("Total delay: ", _.totalDelay)
+ showMillisDistribution("Processing time: ", _.processingDelay)
+ }
+
+ def showMillisDistribution(heading: String, getMetric: BatchInfo => Option[Long]) {
+ org.apache.spark.scheduler.StatsReportListener.showMillisDistribution(
+ heading, extractDistribution(getMetric))
+ }
+
+ def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = {
+ Distribution(batchInfos.flatMap(getMetric(_)).map(_.toDouble))
+ }
+}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala
new file mode 100644
index 0000000000..110a20f282
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming.scheduler
+
+import org.apache.spark.Logging
+import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import java.util.concurrent.LinkedBlockingQueue
+
+/** Asynchronously passes StreamingListenerEvents to registered StreamingListeners. */
+private[spark] class StreamingListenerBus() extends Logging {
+ private val listeners = new ArrayBuffer[StreamingListener]() with SynchronizedBuffer[StreamingListener]
+
+ /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than
+ * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */
+ private val EVENT_QUEUE_CAPACITY = 10000
+ private val eventQueue = new LinkedBlockingQueue[StreamingListenerEvent](EVENT_QUEUE_CAPACITY)
+ private var queueFullErrorMessageLogged = false
+
+ new Thread("StreamingListenerBus") {
+ setDaemon(true)
+ override def run() {
+ while (true) {
+ val event = eventQueue.take
+ event match {
+ case batchStarted: StreamingListenerBatchStarted =>
+ listeners.foreach(_.onBatchStarted(batchStarted))
+ case batchCompleted: StreamingListenerBatchCompleted =>
+ listeners.foreach(_.onBatchCompleted(batchCompleted))
+ case _ =>
+ }
+ }
+ }
+ }.start()
+
+ def addListener(listener: StreamingListener) {
+ listeners += listener
+ }
+
+ def post(event: StreamingListenerEvent) {
+ val eventAdded = eventQueue.offer(event)
+ if (!eventAdded && !queueFullErrorMessageLogged) {
+ logError("Dropping SparkListenerEvent because no remaining room in event queue. " +
+ "This likely means one of the SparkListeners is too slow and cannot keep up with the " +
+ "rate at which tasks are being started by the scheduler.")
+ 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/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
index 6977957126..1559f7a9f7 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
@@ -25,6 +25,7 @@ import StreamingContext._
import scala.util.Random
import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import scala.reflect.ClassTag
import java.io.{File, ObjectInputStream, IOException}
import java.util.UUID
@@ -38,7 +39,6 @@ import org.apache.hadoop.conf.Configuration
private[streaming]
object MasterFailureTest extends Logging {
- initLogging()
@volatile var killed = false
@volatile var killCount = 0
@@ -120,7 +120,7 @@ object MasterFailureTest extends Logging {
* Tests stream operation with multiple master failures, and verifies whether the
* final set of output values is as expected or not.
*/
- def testOperation[T: ClassManifest](
+ def testOperation[T: ClassTag](
directory: String,
batchDuration: Duration,
input: Seq[String],
@@ -158,7 +158,7 @@ object MasterFailureTest extends Logging {
* and batch duration. Returns the streaming context and the directory to which
* files should be written for testing.
*/
- private def setupStreams[T: ClassManifest](
+ private def setupStreams[T: ClassTag](
directory: String,
batchDuration: Duration,
operation: DStream[String] => DStream[T]
@@ -192,7 +192,7 @@ object MasterFailureTest extends Logging {
* Repeatedly starts and kills the streaming context until timed out or
* the last expected output is generated. Finally, return
*/
- private def runStreams[T: ClassManifest](
+ private def runStreams[T: ClassTag](
ssc_ : StreamingContext,
lastExpectedOutput: T,
maxTimeToRun: Long
@@ -274,7 +274,7 @@ object MasterFailureTest extends Logging {
* duplicate batch outputs of values from the `output`. As a result, the
* expected output should not have consecutive batches with the same values as output.
*/
- private def verifyOutput[T: ClassManifest](output: Seq[T], expectedOutput: Seq[T]) {
+ private def verifyOutput[T: ClassTag](output: Seq[T], expectedOutput: Seq[T]) {
// Verify whether expected outputs do not consecutive batches with same output
for (i <- 0 until expectedOutput.size - 1) {
assert(expectedOutput(i) != expectedOutput(i+1),
@@ -305,7 +305,7 @@ object MasterFailureTest extends Logging {
* ArrayBuffer. This buffer is wiped clean on being restored from checkpoint.
*/
private[streaming]
-class TestOutputStream[T: ClassManifest](
+class TestOutputStream[T: ClassTag](
parent: DStream[T],
val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]]
) extends ForEachDStream[T](
@@ -330,7 +330,6 @@ class TestOutputStream[T: ClassManifest](
*/
private[streaming]
class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread with Logging {
- initLogging()
override def run() {
try {
@@ -365,7 +364,6 @@ class KillingThread(ssc: StreamingContext, maxKillWaitTime: Long) extends Thread
private[streaming]
class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
extends Thread with Logging {
- initLogging()
override def run() {
val localTestDir = Files.createTempDir()
@@ -380,24 +378,24 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString)
FileUtils.writeStringToFile(localFile, input(i).toString + "\n")
var tries = 0
- var done = false
- while (!done && tries < maxTries) {
- tries += 1
- try {
- // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
- fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile)
- fs.rename(tempHadoopFile, hadoopFile)
- done = true
- } catch {
- case ioe: IOException => {
- fs = testDir.getFileSystem(new Configuration())
- logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe)
- }
- }
+ var done = false
+ while (!done && tries < maxTries) {
+ tries += 1
+ try {
+ // fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
+ fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile)
+ fs.rename(tempHadoopFile, hadoopFile)
+ done = true
+ } catch {
+ case ioe: IOException => {
+ fs = testDir.getFileSystem(new Configuration())
+ logWarning("Attempt " + tries + " at generating file " + hadoopFile + " failed.", ioe)
+ }
+ }
}
- if (!done)
+ if (!done)
logError("Could not generate file " + hadoopFile)
- else
+ else
logInfo("Generated file " + hadoopFile + " at " + System.currentTimeMillis)
Thread.sleep(interval)
localFile.delete()
@@ -411,5 +409,3 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
}
}
}
-
-
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index fc8655a083..6585d494a6 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -20,7 +20,7 @@ package org.apache.spark.streaming.util
import java.nio.ByteBuffer
import org.apache.spark.util.{RateLimitedOutputStream, IntParam}
import java.net.ServerSocket
-import org.apache.spark.{Logging}
+import org.apache.spark.{SparkConf, Logging}
import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
import scala.io.Source
import java.io.IOException
@@ -42,7 +42,7 @@ object RawTextSender extends Logging {
// Repeat the input data multiple times to fill in a buffer
val lines = Source.fromFile(file).getLines().toArray
val bufferStream = new FastByteArrayOutputStream(blockSize + 1000)
- val ser = new KryoSerializer().newInstance()
+ val ser = new KryoSerializer(new SparkConf()).newInstance()
val serStream = ser.serializeStream(bufferStream)
var i = 0
while (bufferStream.position < blockSize) {
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index ad4a8b9535..0d2145da9a 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -17,63 +17,33 @@
package org.apache.spark.streaming;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.io.Files;
-import kafka.serializer.StringDecoder;
-import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
-import org.apache.spark.streaming.api.java.JavaDStreamLike;
+import scala.Tuple2;
+
import org.junit.After;
import org.junit.Assert;
-import org.junit.Before;
import org.junit.Test;
-import scala.Tuple2;
+import java.io.*;
+import java.util.*;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.io.Files;
+
+import org.apache.spark.SparkConf;
import org.apache.spark.HashPartitioner;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaRDDLike;
-import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.*;
import org.apache.spark.storage.StorageLevel;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
-import org.apache.spark.streaming.JavaTestUtils;
-import org.apache.spark.streaming.JavaCheckpointTestUtils;
-import org.apache.spark.streaming.InputStreamsSuite;
-
-import java.io.*;
-import java.util.*;
-
-import akka.actor.Props;
-import akka.zeromq.Subscribe;
-
-
// The test suite itself is Serializable so that anonymous Function implementations can be
// serialized, as an alternative to converting these anonymous classes to static inner classes;
// see http://stackoverflow.com/questions/758570/.
-public class JavaAPISuite implements Serializable {
- private transient JavaStreamingContext ssc;
-
- @Before
- public void setUp() {
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
- ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
- ssc.checkpoint("checkpoint");
- }
-
- @After
- public void tearDown() {
- ssc.stop();
- ssc = null;
-
- // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
- System.clearProperty("spark.driver.port");
- }
-
+public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable {
@Test
public void testCount() {
List<List<Integer>> inputData = Arrays.asList(
@@ -86,8 +56,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(3L),
Arrays.asList(1L));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream count = stream.count();
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Long> count = stream.count();
JavaTestUtils.attachTestOutputStream(count);
List<List<Long>> result = JavaTestUtils.runStreams(ssc, 3, 3);
assertOrderInvariantEquals(expected, result);
@@ -99,12 +69,12 @@ public class JavaAPISuite implements Serializable {
Arrays.asList("hello", "world"),
Arrays.asList("goodnight", "moon"));
- List<List<Integer>> expected = Arrays.asList(
+ List<List<Integer>> expected = Arrays.asList(
Arrays.asList(5,5),
Arrays.asList(9,4));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
@Override
public Integer call(String s) throws Exception {
return s.length();
@@ -129,8 +99,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(7,8,9,4,5,6),
Arrays.asList(7,8,9));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream windowed = stream.window(new Duration(2000));
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> windowed = stream.window(new Duration(2000));
JavaTestUtils.attachTestOutputStream(windowed);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
@@ -153,8 +123,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(7,8,9,10,11,12,13,14,15,16,17,18),
Arrays.asList(13,14,15,16,17,18));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream windowed = stream.window(new Duration(4000), new Duration(2000));
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> windowed = stream.window(new Duration(4000), new Duration(2000));
JavaTestUtils.attachTestOutputStream(windowed);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 8, 4);
@@ -171,8 +141,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList("giants"),
Arrays.asList("yankees"));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream filtered = stream.filter(new Function<String, Boolean>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> filtered = stream.filter(new Function<String, Boolean>() {
@Override
public Boolean call(String s) throws Exception {
return s.contains("a");
@@ -227,8 +197,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(Arrays.asList("giants", "dodgers")),
Arrays.asList(Arrays.asList("yankees", "red socks")));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream glommed = stream.glom();
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<List<String>> glommed = stream.glom();
JavaTestUtils.attachTestOutputStream(glommed);
List<List<List<String>>> result = JavaTestUtils.runStreams(ssc, 2, 2);
@@ -245,8 +215,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList("GIANTSDODGERS"),
Arrays.asList("YANKEESRED SOCKS"));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream mapped = stream.mapPartitions(new FlatMapFunction<Iterator<String>, String>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> mapped = stream.mapPartitions(new FlatMapFunction<Iterator<String>, String>() {
@Override
public Iterable<String> call(Iterator<String> in) {
String out = "";
@@ -288,8 +258,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(15),
Arrays.asList(24));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream reduced = stream.reduce(new IntegerSum());
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> reduced = stream.reduce(new IntegerSum());
JavaTestUtils.attachTestOutputStream(reduced);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
@@ -309,8 +279,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(39),
Arrays.asList(24));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream reducedWindowed = stream.reduceByWindow(new IntegerSum(),
+ JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> reducedWindowed = stream.reduceByWindow(new IntegerSum(),
new IntegerDifference(), new Duration(2000), new Duration(1000));
JavaTestUtils.attachTestOutputStream(reducedWindowed);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 4, 4);
@@ -695,8 +665,8 @@ public class JavaAPISuite implements Serializable {
Arrays.asList("b", "o", "o", "d","o","d","g","e","r","s"),
Arrays.asList("a","t","h","l","e","t","i","c","s"));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> flatMapped = stream.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
return Lists.newArrayList(x.split("(?!^)"));
@@ -742,8 +712,8 @@ public class JavaAPISuite implements Serializable {
new Tuple2<Integer, String>(9, "c"),
new Tuple2<Integer, String>(9, "s")));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction<String, Integer, String>() {
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaPairDStream<Integer,String> flatMapped = stream.flatMap(new PairFlatMapFunction<String, Integer, String>() {
@Override
public Iterable<Tuple2<Integer, String>> call(String in) throws Exception {
List<Tuple2<Integer, String>> out = Lists.newArrayList();
@@ -776,10 +746,10 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(2,2,5,5),
Arrays.asList(3,3,6,6));
- JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
- JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
+ JavaDStream<Integer> stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 2);
+ JavaDStream<Integer> stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 2);
- JavaDStream unioned = stream1.union(stream2);
+ JavaDStream<Integer> unioned = stream1.union(stream2);
JavaTestUtils.attachTestOutputStream(unioned);
List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3);
@@ -790,7 +760,7 @@ public class JavaAPISuite implements Serializable {
* Performs an order-invariant comparison of lists representing two RDD streams. This allows
* us to account for ordering variation within individual RDD's which occurs during windowing.
*/
- public static <T extends Comparable> void assertOrderInvariantEquals(
+ public static <T extends Comparable<T>> void assertOrderInvariantEquals(
List<List<T>> expected, List<List<T>> actual) {
for (List<T> list: expected) {
Collections.sort(list);
@@ -813,11 +783,11 @@ public class JavaAPISuite implements Serializable {
Arrays.asList(new Tuple2<String, Integer>("giants", 6)),
Arrays.asList(new Tuple2<String, Integer>("yankees", 7)));
- JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1);
JavaPairDStream<String, Integer> pairStream = stream.map(
new PairFunction<String, String, Integer>() {
@Override
- public Tuple2 call(String in) throws Exception {
+ public Tuple2<String, Integer> call(String in) throws Exception {
return new Tuple2<String, Integer>(in, in.length());
}
});
@@ -1540,8 +1510,8 @@ public class JavaAPISuite implements Serializable {
File tempDir = Files.createTempDir();
ssc.checkpoint(tempDir.getAbsolutePath());
- JavaDStream stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
- JavaDStream letterCount = stream.map(new Function<String, Integer>() {
+ JavaDStream<String> stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1);
+ JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() {
@Override
public Integer call(String s) throws Exception {
return s.length();
@@ -1595,28 +1565,8 @@ public class JavaAPISuite implements Serializable {
// Java arguments and assign it to a JavaDStream without producing type errors. Testing of the
// InputStream functionality is deferred to the existing Scala tests.
@Test
- public void testKafkaStream() {
- HashMap<String, Integer> topics = Maps.newHashMap();
- JavaPairDStream<String, String> test1 = ssc.kafkaStream("localhost:12345", "group", topics);
- JavaPairDStream<String, String> test2 = ssc.kafkaStream("localhost:12345", "group", topics,
- StorageLevel.MEMORY_AND_DISK());
-
- HashMap<String, String> kafkaParams = Maps.newHashMap();
- kafkaParams.put("zookeeper.connect","localhost:12345");
- kafkaParams.put("group.id","consumer-group");
- JavaPairDStream<String, String> test3 = ssc.kafkaStream(
- String.class,
- String.class,
- StringDecoder.class,
- StringDecoder.class,
- kafkaParams,
- topics,
- StorageLevel.MEMORY_AND_DISK());
- }
-
- @Test
public void testSocketTextStream() {
- JavaDStream test = ssc.socketTextStream("localhost", 12345);
+ JavaDStream<String> test = ssc.socketTextStream("localhost", 12345);
}
@Test
@@ -1636,7 +1586,7 @@ public class JavaAPISuite implements Serializable {
}
}
- JavaDStream test = ssc.socketStream(
+ JavaDStream<String> test = ssc.socketStream(
"localhost",
12345,
new Converter(),
@@ -1645,43 +1595,11 @@ public class JavaAPISuite implements Serializable {
@Test
public void testTextFileStream() {
- JavaDStream test = ssc.textFileStream("/tmp/foo");
+ JavaDStream<String> test = ssc.textFileStream("/tmp/foo");
}
@Test
public void testRawSocketStream() {
- JavaDStream test = ssc.rawSocketStream("localhost", 12345);
- }
-
- @Test
- public void testFlumeStream() {
- JavaDStream test = ssc.flumeStream("localhost", 12345, StorageLevel.MEMORY_ONLY());
- }
-
- @Test
- public void testFileStream() {
- JavaPairDStream<String, String> foo =
- ssc.<String, String, SequenceFileInputFormat>fileStream("/tmp/foo");
- }
-
- @Test
- public void testTwitterStream() {
- String[] filters = new String[] { "good", "bad", "ugly" };
- JavaDStream test = ssc.twitterStream(filters, StorageLevel.MEMORY_ONLY());
- }
-
- @Test
- public void testActorStream() {
- JavaDStream test = ssc.actorStream((Props)null, "TestActor", StorageLevel.MEMORY_ONLY());
- }
-
- @Test
- public void testZeroMQStream() {
- JavaDStream test = ssc.zeroMQStream("url", (Subscribe) null, new Function<byte[][], Iterable<String>>() {
- @Override
- public Iterable<String> call(byte[][] b) throws Exception {
- return null;
- }
- });
+ JavaDStream<String> test = ssc.rawSocketStream("localhost", 12345);
}
}
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
index 5e384eeee4..42ab9590d6 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
@@ -17,7 +17,9 @@
package org.apache.spark.streaming
-import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
+import scala.reflect.ClassTag
+
import java.util.{List => JList}
import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaDStreamLike, JavaDStream, JavaStreamingContext}
import org.apache.spark.streaming._
@@ -31,15 +33,15 @@ trait JavaTestBase extends TestSuiteBase {
/**
* Create a [[org.apache.spark.streaming.TestInputStream]] and attach it to the supplied context.
* The stream will be derived from the supplied lists of Java objects.
- **/
+ */
def attachTestInputStream[T](
ssc: JavaStreamingContext,
data: JList[JList[T]],
numPartitions: Int) = {
val seqData = data.map(Seq(_:_*))
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val dstream = new TestInputStream[T](ssc.ssc, seqData, numPartitions)
ssc.ssc.registerInputStream(dstream)
new JavaDStream[T](dstream)
@@ -52,8 +54,8 @@ trait JavaTestBase extends TestSuiteBase {
def attachTestOutputStream[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]](
dstream: JavaDStreamLike[T, This, R]) =
{
- implicit val cm: ClassManifest[T] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
+ implicit val cm: ClassTag[T] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val ostream = new TestOutputStreamWithPartitions(dstream.dstream)
dstream.dstream.ssc.registerOutputStream(ostream)
}
@@ -67,8 +69,8 @@ trait JavaTestBase extends TestSuiteBase {
*/
def runStreams[V](
ssc: JavaStreamingContext, numBatches: Int, numExpectedOutput: Int): JList[JList[V]] = {
- implicit val cm: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cm: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
val res = runStreams[V](ssc.ssc, numBatches, numExpectedOutput)
val out = new ArrayList[JList[V]]()
res.map(entry => out.append(new ArrayList[V](entry)))
@@ -85,8 +87,8 @@ trait JavaTestBase extends TestSuiteBase {
*/
def runStreamsWithPartitions[V](ssc: JavaStreamingContext, numBatches: Int,
numExpectedOutput: Int): JList[JList[JList[V]]] = {
- implicit val cm: ClassManifest[V] =
- implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[V]]
+ implicit val cm: ClassTag[V] =
+ implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]]
val res = runStreamsWithPartitions[V](ssc.ssc, numBatches, numExpectedOutput)
val out = new ArrayList[JList[JList[V]]]()
res.map{entry =>
diff --git a/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
new file mode 100644
index 0000000000..34bee56885
--- /dev/null
+++ b/streaming/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java
@@ -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.streaming;
+
+import org.apache.spark.streaming.api.java.JavaStreamingContext;
+import org.junit.After;
+import org.junit.Before;
+
+public abstract class LocalJavaStreamingContext {
+
+ protected transient JavaStreamingContext ssc;
+
+ @Before
+ public void setUp() {
+ System.clearProperty("spark.driver.port");
+ System.clearProperty("spark.hostPort");
+ System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock");
+ ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000));
+ ssc.checkpoint("checkpoint");
+ }
+
+ @After
+ public void tearDown() {
+ ssc.stop();
+ ssc = null;
+
+ // 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/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index 259ef1608c..ee6b433d1f 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -23,21 +23,9 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
import util.ManualClock
+import org.apache.spark.{SparkContext, SparkConf}
class BasicOperationsSuite extends TestSuiteBase {
-
- override def framework() = "BasicOperationsSuite"
-
- before {
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
- }
-
- after {
- // 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")
- }
-
test("map") {
val input = Seq(1 to 4, 5 to 8, 9 to 12)
testOperation(
@@ -387,7 +375,11 @@ class BasicOperationsSuite extends TestSuiteBase {
}
test("slice") {
- val ssc = new StreamingContext("local[2]", "BasicOperationSuite", Seconds(1))
+ val conf2 = new SparkConf()
+ .setMaster("local[2]")
+ .setAppName("BasicOperationsSuite")
+ .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ val ssc = new StreamingContext(new SparkContext(conf2), Seconds(1))
val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
val stream = new TestInputStream[Int](ssc, input, 2)
ssc.registerInputStream(stream)
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index beb20831bd..8dc80ac2ed 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -17,56 +17,50 @@
package org.apache.spark.streaming
-import dstream.FileInputDStream
-import org.apache.spark.streaming.StreamingContext._
import java.io.File
-import runtime.RichInt
-import org.scalatest.BeforeAndAfter
+
+import scala.collection.mutable.ArrayBuffer
+import scala.reflect.ClassTag
import org.apache.commons.io.FileUtils
-import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
-import util.{Clock, ManualClock}
-import scala.util.Random
import com.google.common.io.Files
-
+import org.apache.hadoop.fs.{Path, FileSystem}
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.streaming.dstream.FileInputDStream
+import org.apache.spark.streaming.util.ManualClock
/**
* This test suites tests the checkpointing functionality of DStreams -
* the checkpointing of a DStream's RDDs as well as the checkpointing of
* the whole DStream graph.
*/
-class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
+class CheckpointSuite extends TestSuiteBase {
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ var ssc: StreamingContext = null
- before {
+ override def batchDuration = Milliseconds(500)
+
+ override def actuallyWait = true // to allow checkpoints to be written
+
+ override def beforeFunction() {
+ super.beforeFunction()
FileUtils.deleteDirectory(new File(checkpointDir))
}
- after {
+ override def afterFunction() {
+ super.afterFunction()
if (ssc != null) ssc.stop()
FileUtils.deleteDirectory(new File(checkpointDir))
-
- // 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")
}
- var ssc: StreamingContext = null
-
- override def framework = "CheckpointSuite"
-
- override def batchDuration = Milliseconds(500)
-
- override def actuallyWait = true
-
test("basic rdd checkpoints + dstream graph checkpoint recovery") {
assert(batchDuration === Milliseconds(500), "batchDuration for this test must be 1 second")
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
val stateStreamCheckpointInterval = Seconds(1)
-
+ val fs = FileSystem.getLocal(new Configuration())
// this ensure checkpointing occurs at least once
val firstNumBatches = (stateStreamCheckpointInterval / batchDuration).toLong * 2
val secondNumBatches = firstNumBatches
@@ -74,13 +68,13 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
// Setup the streams
val input = (1 to 10).map(_ => Seq("a")).toSeq
val operation = (st: DStream[String]) => {
- val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
- Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+ Some((values.foldLeft(0)(_ + _) + state.getOrElse(0)))
}
st.map(x => (x, 1))
- .updateStateByKey[RichInt](updateFunc)
+ .updateStateByKey(updateFunc)
.checkpoint(stateStreamCheckpointInterval)
- .map(t => (t._1, t._2.self))
+ .map(t => (t._1, t._2))
}
var ssc = setupStreams(input, operation)
var stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
@@ -90,11 +84,12 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
ssc.start()
advanceTimeWithRealDelay(ssc, firstNumBatches)
logInfo("Checkpoint data of state stream = \n" + stateStream.checkpointData)
- assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before first failure")
+ assert(!stateStream.checkpointData.checkpointFiles.isEmpty,
+ "No checkpointed RDDs in state stream before first failure")
stateStream.checkpointData.checkpointFiles.foreach {
- case (time, data) => {
- val file = new File(data.toString)
- assert(file.exists(), "Checkpoint file '" + file +"' for time " + time + " for state stream before first failure does not exist")
+ case (time, file) => {
+ assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time +
+ " for state stream before first failure does not exist")
}
}
@@ -102,7 +97,8 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
// and check whether the earlier checkpoint files are deleted
val checkpointFiles = stateStream.checkpointData.checkpointFiles.map(x => new File(x._2))
advanceTimeWithRealDelay(ssc, secondNumBatches)
- checkpointFiles.foreach(file => assert(!file.exists, "Checkpoint file '" + file + "' was not deleted"))
+ checkpointFiles.foreach(file =>
+ assert(!file.exists, "Checkpoint file '" + file + "' was not deleted"))
ssc.stop()
// Restart stream computation using the checkpoint file and check whether
@@ -110,19 +106,20 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
ssc = new StreamingContext(checkpointDir)
stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
- assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from first failure")
+ assert(!stateStream.generatedRDDs.isEmpty,
+ "No restored RDDs in state stream after recovery from first failure")
// Run one batch to generate a new checkpoint file and check whether some RDD
// is present in the checkpoint data or not
ssc.start()
advanceTimeWithRealDelay(ssc, 1)
- assert(!stateStream.checkpointData.checkpointFiles.isEmpty, "No checkpointed RDDs in state stream before second failure")
+ assert(!stateStream.checkpointData.checkpointFiles.isEmpty,
+ "No checkpointed RDDs in state stream before second failure")
stateStream.checkpointData.checkpointFiles.foreach {
- case (time, data) => {
- val file = new File(data.toString)
- assert(file.exists(),
- "Checkpoint file '" + file +"' for time " + time + " for state stream before seconds failure does not exist")
+ case (time, file) => {
+ assert(fs.exists(new Path(file)), "Checkpoint file '" + file +"' for time " + time +
+ " for state stream before seconds failure does not exist")
}
}
ssc.stop()
@@ -132,10 +129,12 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
ssc = new StreamingContext(checkpointDir)
stateStream = ssc.graph.getOutputStreams().head.dependencies.head.dependencies.head
logInfo("Restored data of state stream = \n[" + stateStream.generatedRDDs.mkString("\n") + "]")
- assert(!stateStream.generatedRDDs.isEmpty, "No restored RDDs in state stream after recovery from second failure")
+ assert(!stateStream.generatedRDDs.isEmpty,
+ "No restored RDDs in state stream after recovery from second failure")
- // Adjust manual clock time as if it is being restarted after a delay
- System.setProperty("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
+ // Adjust manual clock time as if it is being restarted after a delay; this is a hack because
+ // we modify the conf object, but it works for this one property
+ ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration.milliseconds * 7).toString)
ssc.start()
advanceTimeWithRealDelay(ssc, 4)
ssc.stop()
@@ -143,6 +142,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
ssc = null
}
+
// This tests whether the systm can recover from a master failure with simple
// non-stateful operations. This assumes as reliable, replayable input
// source - TestInputDStream.
@@ -180,26 +180,23 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
val input = (1 to 10).map(_ => Seq("a")).toSeq
val output = (1 to 10).map(x => Seq(("a", x))).toSeq
val operation = (st: DStream[String]) => {
- val updateFunc = (values: Seq[Int], state: Option[RichInt]) => {
- Some(new RichInt(values.foldLeft(0)(_ + _) + state.map(_.self).getOrElse(0)))
+ val updateFunc = (values: Seq[Int], state: Option[Int]) => {
+ Some((values.foldLeft(0)(_ + _) + state.getOrElse(0)))
}
st.map(x => (x, 1))
- .updateStateByKey[RichInt](updateFunc)
+ .updateStateByKey(updateFunc)
.checkpoint(batchDuration * 2)
- .map(t => (t._1, t._2.self))
+ .map(t => (t._1, t._2))
}
testCheckpointedOperation(input, operation, output, 7)
}
+
// This tests whether file input stream remembers what files were seen before
// the master failure and uses them again to process a large window operation.
// It also tests whether batches, whose processing was incomplete due to the
// failure, are re-processed or not.
test("recovery with file input stream") {
- // Disable manual clock as FileInputDStream does not work with manual clock
- val clockProperty = System.getProperty("spark.streaming.clock")
- System.clearProperty("spark.streaming.clock")
-
// Set up the streaming context and input streams
val testDir = Files.createTempDir()
var ssc = new StreamingContext(master, framework, Seconds(1))
@@ -296,10 +293,6 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
)
// To ensure that all the inputs were received correctly
assert(expectedOutput.last === output.last)
-
- // Enable manual clock back again for other tests
- if (clockProperty != null)
- System.setProperty("spark.streaming.clock", clockProperty)
}
@@ -312,7 +305,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
* NOTE: This takes into consideration that the last batch processed before
* master failure will be re-processed after restart/recovery.
*/
- def testCheckpointedOperation[U: ClassManifest, V: ClassManifest](
+ def testCheckpointedOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
@@ -356,7 +349,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
* Advances the manual clock on the streaming scheduler by given number of batches.
* It also waits for the expected amount of time for each batch.
*/
- def advanceTimeWithRealDelay[V: ClassManifest](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = {
+ def advanceTimeWithRealDelay[V: ClassTag](ssc: StreamingContext, numBatches: Long): Seq[Seq[V]] = {
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
logInfo("Manual clock before advancing = " + clock.time)
for (i <- 1 to numBatches.toInt) {
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala
index 6337c5359c..da9b04de1a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala
@@ -32,17 +32,22 @@ import collection.mutable.ArrayBuffer
* This testsuite tests master failures at random times while the stream is running using
* the real clock.
*/
-class FailureSuite extends FunSuite with BeforeAndAfter with Logging {
+class FailureSuite extends TestSuiteBase with Logging {
var directory = "FailureSuite"
val numBatches = 30
- val batchDuration = Milliseconds(1000)
- before {
+ override def batchDuration = Milliseconds(1000)
+
+ override def useManualClock = false
+
+ override def beforeFunction() {
+ super.beforeFunction()
FileUtils.deleteDirectory(new File(directory))
}
- after {
+ override def afterFunction() {
+ super.afterFunction()
FileUtils.deleteDirectory(new File(directory))
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index 7dc82decef..a8e053278a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -23,7 +23,7 @@ import akka.actor.IOManager
import akka.actor.Props
import akka.util.ByteString
-import org.apache.spark.streaming.dstream.{NetworkReceiver, SparkFlumeEvent}
+import org.apache.spark.streaming.dstream.{NetworkReceiver}
import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket}
import java.io.{File, BufferedWriter, OutputStreamWriter}
import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue}
@@ -31,18 +31,11 @@ import collection.mutable.{SynchronizedBuffer, ArrayBuffer}
import util.ManualClock
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.receivers.Receiver
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.Logging
import scala.util.Random
import org.apache.commons.io.FileUtils
import org.scalatest.BeforeAndAfter
-import org.apache.flume.source.avro.AvroSourceProtocol
-import org.apache.flume.source.avro.AvroFlumeEvent
-import org.apache.flume.source.avro.Status
-import org.apache.avro.ipc.{specific, NettyTransceiver}
-import org.apache.avro.ipc.specific.SpecificRequestor
-import java.nio.ByteBuffer
import collection.JavaConversions._
-import java.nio.charset.Charset
import com.google.common.io.Files
import java.util.concurrent.atomic.AtomicInteger
@@ -50,27 +43,15 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
val testPort = 9999
- override def checkpointDir = "checkpoint"
-
- before {
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
- }
-
- after {
- // 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")
- }
-
test("socket input stream") {
// Start the server
val testServer = new TestServer()
testServer.start()
// Set up the streaming context and input streams
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK)
- val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]]
+ val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
val outputStream = new TestOutputStream(networkStream, outputBuffer)
def output = outputBuffer.flatMap(x => x)
ssc.registerOutputStream(outputStream)
@@ -111,62 +92,13 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
}
- test("flume input stream") {
- // Set up the streaming context and input streams
- val ssc = new StreamingContext(master, framework, batchDuration)
- val flumeStream = ssc.flumeStream("localhost", testPort, StorageLevel.MEMORY_AND_DISK)
- val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
- with SynchronizedBuffer[Seq[SparkFlumeEvent]]
- val outputStream = new TestOutputStream(flumeStream, outputBuffer)
- ssc.registerOutputStream(outputStream)
- ssc.start()
-
- val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
- val input = Seq(1, 2, 3, 4, 5)
- Thread.sleep(1000)
- val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort))
- val client = SpecificRequestor.getClient(
- classOf[AvroSourceProtocol], transceiver)
-
- for (i <- 0 until input.size) {
- val event = new AvroFlumeEvent
- event.setBody(ByteBuffer.wrap(input(i).toString.getBytes()))
- event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header"))
- client.append(event)
- Thread.sleep(500)
- clock.addToTime(batchDuration.milliseconds)
- }
-
- val startTime = System.currentTimeMillis()
- while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
- logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size)
- Thread.sleep(100)
- }
- Thread.sleep(1000)
- val timeTaken = System.currentTimeMillis() - startTime
- assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
- logInfo("Stopping context")
- ssc.stop()
-
- val decoder = Charset.forName("UTF-8").newDecoder()
-
- assert(outputBuffer.size === input.length)
- for (i <- 0 until outputBuffer.size) {
- assert(outputBuffer(i).size === 1)
- val str = decoder.decode(outputBuffer(i).head.event.getBody)
- assert(str.toString === input(i).toString)
- assert(outputBuffer(i).head.event.getHeaders.get("test") === "header")
- }
- }
-
-
test("file input stream") {
// Disable manual clock as FileInputDStream does not work with manual clock
- System.clearProperty("spark.streaming.clock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
// Set up the streaming context and input streams
val testDir = Files.createTempDir()
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val fileStream = ssc.textFileStream(testDir.toString)
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
def output = outputBuffer.flatMap(x => x)
@@ -207,7 +139,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
FileUtils.deleteDirectory(testDir)
// Enable manual clock back again for other tests
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
}
@@ -218,7 +150,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
testServer.start()
// Set up the streaming context and input streams
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor",
StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
@@ -261,20 +193,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
}
}
- test("kafka input stream") {
- val ssc = new StreamingContext(master, framework, batchDuration)
- val topics = Map("my-topic" -> 1)
- val test1 = ssc.kafkaStream("localhost:12345", "group", topics)
- val test2 = ssc.kafkaStream("localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK)
-
- // Test specifying decoder
- val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group")
- val test3 = ssc.kafkaStream[
- String,
- String,
- kafka.serializer.StringDecoder,
- kafka.serializer.StringDecoder](kafkaParams, topics, StorageLevel.MEMORY_AND_DISK)
- }
test("multi-thread receiver") {
// set up the test receiver
@@ -285,7 +203,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
MultiThreadTestReceiver.haveAllThreadsFinished = false
// set up the network stream using the test receiver
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
val networkStream = ssc.networkStream[Int](testReceiver)
val countStream = networkStream.count
val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]]
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
new file mode 100644
index 0000000000..fa64142096
--- /dev/null
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.streaming
+
+import org.apache.spark.streaming.scheduler._
+import scala.collection.mutable.ArrayBuffer
+import org.scalatest.matchers.ShouldMatchers
+
+class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers {
+
+ val input = (1 to 4).map(Seq(_)).toSeq
+ val operation = (d: DStream[Int]) => d.map(x => x)
+
+ // To make sure that the processing start and end times in collected
+ // information are different for successive batches
+ override def batchDuration = Milliseconds(100)
+ override def actuallyWait = true
+
+ test("basic BatchInfo generation") {
+ val ssc = setupStreams(input, operation)
+ val collector = new BatchInfoCollector
+ ssc.addStreamingListener(collector)
+ runStreams(ssc, input.size, input.size)
+ val batchInfos = collector.batchInfos
+ batchInfos should have size 4
+
+ batchInfos.foreach(info => {
+ info.schedulingDelay should not be None
+ info.processingDelay should not be None
+ info.totalDelay should not be None
+ info.schedulingDelay.get should be >= 0L
+ info.processingDelay.get should be >= 0L
+ info.totalDelay.get should be >= 0L
+ })
+
+ isInIncreasingOrder(batchInfos.map(_.submissionTime)) should be (true)
+ isInIncreasingOrder(batchInfos.map(_.processingStartTime.get)) should be (true)
+ isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true)
+ }
+
+ /** Check if a sequence of numbers is in increasing order */
+ def isInIncreasingOrder(seq: Seq[Long]): Boolean = {
+ for(i <- 1 until seq.size) {
+ if (seq(i - 1) > seq(i)) return false
+ }
+ true
+ }
+
+ /** Listener that collects information on processed batches */
+ class BatchInfoCollector extends StreamingListener {
+ val batchInfos = new ArrayBuffer[BatchInfo]
+ override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) {
+ batchInfos += batchCompleted.batchInfo
+ }
+ }
+}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 8c8c359e6e..b20d02f996 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -20,14 +20,15 @@ package org.apache.spark.streaming
import org.apache.spark.streaming.dstream.{InputDStream, ForEachDStream}
import org.apache.spark.streaming.util.ManualClock
-import collection.mutable.ArrayBuffer
-import collection.mutable.SynchronizedBuffer
+import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.SynchronizedBuffer
+import scala.reflect.ClassTag
import java.io.{ObjectInputStream, IOException}
import org.scalatest.{BeforeAndAfter, FunSuite}
-import org.apache.spark.Logging
+import org.apache.spark.{SparkContext, SparkConf, Logging}
import org.apache.spark.rdd.RDD
/**
@@ -35,7 +36,7 @@ import org.apache.spark.rdd.RDD
* replayable, reliable message queue like Kafka. It requires a sequence as input, and
* returns the i_th element at the i_th batch unde manual clock.
*/
-class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
+class TestInputStream[T: ClassTag](ssc_ : StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
extends InputDStream[T](ssc_) {
def start() {}
@@ -63,7 +64,7 @@ class TestInputStream[T: ClassManifest](ssc_ : StreamingContext, input: Seq[Seq[
*
* The buffer contains a sequence of RDD's, each containing a sequence of items
*/
-class TestOutputStream[T: ClassManifest](parent: DStream[T],
+class TestOutputStream[T: ClassTag](parent: DStream[T],
val output: ArrayBuffer[Seq[T]] = ArrayBuffer[Seq[T]]())
extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
val collected = rdd.collect()
@@ -85,7 +86,7 @@ class TestOutputStream[T: ClassManifest](parent: DStream[T],
* The buffer contains a sequence of RDD's, each containing a sequence of partitions, each
* containing a sequence of items.
*/
-class TestOutputStreamWithPartitions[T: ClassManifest](parent: DStream[T],
+class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T],
val output: ArrayBuffer[Seq[Seq[T]]] = ArrayBuffer[Seq[Seq[T]]]())
extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
val collected = rdd.glom().collect().map(_.toSeq)
@@ -109,7 +110,7 @@ class TestOutputStreamWithPartitions[T: ClassManifest](parent: DStream[T],
trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
// Name of the framework for Spark context
- def framework = "TestSuiteBase"
+ def framework = this.getClass.getSimpleName
// Master for Spark context
def master = "local[2]"
@@ -126,21 +127,52 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
// Maximum time to wait before the test times out
def maxWaitTimeMillis = 10000
+ // Whether to use manual clock or not
+ def useManualClock = true
+
// Whether to actually wait in real time before changing manual clock
def actuallyWait = false
+ //// A SparkConf to use in tests. Can be modified before calling setupStreams to configure things.
+ val conf = new SparkConf()
+ .setMaster(master)
+ .setAppName(framework)
+ .set("spark.cleaner.ttl", "3600")
+
+ // Default before function for any streaming test suite. Override this
+ // if you want to add your stuff to "before" (i.e., don't call before { } )
+ def beforeFunction() {
+ if (useManualClock) {
+ logInfo("Using manual clock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ } else {
+ logInfo("Using real clock")
+ conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock")
+ }
+ }
+
+ // Default after function for any streaming test suite. Override this
+ // if you want to add your stuff to "after" (i.e., don't call after { } )
+ def afterFunction() {
+ // 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")
+ }
+
+ before(beforeFunction)
+ after(afterFunction)
+
/**
* Set up required DStreams to test the DStream operation using the two sequences
* of input collections.
*/
- def setupStreams[U: ClassManifest, V: ClassManifest](
+ def setupStreams[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
numPartitions: Int = numInputPartitions
): StreamingContext = {
-
// Create StreamingContext
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir)
}
@@ -159,14 +191,13 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Set up required DStreams to test the binary operation using the sequence
* of input collections.
*/
- def setupStreams[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ def setupStreams[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W]
): StreamingContext = {
-
// Create StreamingContext
- val ssc = new StreamingContext(master, framework, batchDuration)
+ val ssc = new StreamingContext(conf, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir)
}
@@ -190,7 +221,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
*
* Returns a sequence of items for each RDD.
*/
- def runStreams[V: ClassManifest](
+ def runStreams[V: ClassTag](
ssc: StreamingContext,
numBatches: Int,
numExpectedOutput: Int
@@ -207,7 +238,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Returns a sequence of RDD's. Each RDD is represented as several sequences of items, each
* representing one partition.
*/
- def runStreamsWithPartitions[V: ClassManifest](
+ def runStreamsWithPartitions[V: ClassTag](
ssc: StreamingContext,
numBatches: Int,
numExpectedOutput: Int
@@ -242,7 +273,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
val startTime = System.currentTimeMillis()
while (output.size < numExpectedOutput && System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput)
- Thread.sleep(100)
+ Thread.sleep(10)
}
val timeTaken = System.currentTimeMillis() - startTime
@@ -263,7 +294,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* is same as the expected output values, by comparing the output
* collections either as lists (order matters) or sets (order does not matter)
*/
- def verifyOutput[V: ClassManifest](
+ def verifyOutput[V: ClassTag](
output: Seq[Seq[V]],
expectedOutput: Seq[Seq[V]],
useSet: Boolean
@@ -293,7 +324,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Test unary DStream operation with a list of inputs, with number of
* batches to run same as the number of expected output values
*/
- def testOperation[U: ClassManifest, V: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
@@ -311,7 +342,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* @param useSet Compare the output values with the expected output values
* as sets (order matters) or as lists (order does not matter)
*/
- def testOperation[U: ClassManifest, V: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
@@ -328,7 +359,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* Test binary DStream operation with two lists of inputs, with number of
* batches to run same as the number of expected output values
*/
- def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W],
@@ -348,7 +379,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
* @param useSet Compare the output values with the expected output values
* as sets (order matters) or as lists (order does not matter)
*/
- def testOperation[U: ClassManifest, V: ClassManifest, W: ClassManifest](
+ def testOperation[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W],
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
index f50e05c0d8..c39abfc21b 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
@@ -18,23 +18,12 @@
package org.apache.spark.streaming
import org.apache.spark.streaming.StreamingContext._
-import collection.mutable.ArrayBuffer
class WindowOperationsSuite extends TestSuiteBase {
- System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
+ override def maxWaitTimeMillis = 20000 // large window tests can sometimes take longer
- override def framework = "WindowOperationsSuite"
-
- override def maxWaitTimeMillis = 20000
-
- override def batchDuration = Seconds(1)
-
- after {
- // 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")
- }
+ override def batchDuration = Seconds(1) // making sure its visible in this class
val largerSlideInput = Seq(
Seq(("a", 1)),
@@ -235,9 +224,7 @@ class WindowOperationsSuite extends TestSuiteBase {
val slideDuration = Seconds(1)
val numBatches = expectedOutput.size * (slideDuration / batchDuration).toInt
val operation = (s: DStream[(String, Int)]) => {
- s.groupByKeyAndWindow(windowDuration, slideDuration)
- .map(x => (x._1, x._2.toSet))
- .persist()
+ s.groupByKeyAndWindow(windowDuration, slideDuration).map(x => (x._1, x._2.toSet))
}
testOperation(input, operation, expectedOutput, numBatches, true)
}
diff --git a/tools/pom.xml b/tools/pom.xml
index f1c489beea..28f5ef14b1 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -25,7 +25,7 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-tools_2.9.3</artifactId>
+ <artifactId>spark-tools_2.10</artifactId>
<packaging>jar</packaging>
<name>Spark Project Tools</name>
<url>http://spark.incubator.apache.org/</url>
@@ -33,24 +33,24 @@
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-streaming_2.9.3</artifactId>
+ <artifactId>spark-streaming_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
index f824c472ae..f670f65bf5 100644
--- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
+++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala
@@ -199,7 +199,7 @@ object JavaAPICompletenessChecker {
private def toJavaMethod(method: SparkMethod): SparkMethod = {
val params = method.parameters
- .filterNot(_.name == "scala.reflect.ClassManifest")
+ .filterNot(_.name == "scala.reflect.ClassTag")
.map(toJavaType(_, isReturnType = false))
SparkMethod(method.name, toJavaType(method.returnType, isReturnType = true), params)
}
@@ -212,7 +212,7 @@ object JavaAPICompletenessChecker {
// internal Spark components.
val excludedNames = Seq(
"org.apache.spark.rdd.RDD.origin",
- "org.apache.spark.rdd.RDD.elementClassManifest",
+ "org.apache.spark.rdd.RDD.elementClassTag",
"org.apache.spark.rdd.RDD.checkpointData",
"org.apache.spark.rdd.RDD.partitioner",
"org.apache.spark.rdd.RDD.partitions",
diff --git a/yarn/README.md b/yarn/README.md
new file mode 100644
index 0000000000..65ee85447e
--- /dev/null
+++ b/yarn/README.md
@@ -0,0 +1,12 @@
+# YARN DIRECTORY LAYOUT
+
+Hadoop Yarn related codes are organized in separate directories to minimize duplicated code.
+
+ * common : Common codes that do not depending on specific version of Hadoop.
+
+ * alpha / stable : Codes that involve specific version of Hadoop YARN API.
+
+ alpha represents 0.23 and 2.0.x
+ stable represents 2.2 and later, until the API changes again.
+
+alpha / stable will build together with common dir into a single jar
diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml
new file mode 100644
index 0000000000..8291e9e7a3
--- /dev/null
+++ b/yarn/alpha/pom.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>yarn-parent_2.10</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-yarn-alpha_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project YARN Alpha API</name>
+
+</project>
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index a7baf0c36c..2bb11e54c5 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -22,9 +22,12 @@ import java.net.Socket
import java.util.concurrent.CopyOnWriteArrayList
import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+import scala.collection.JavaConversions._
+
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.util.ShutdownHookManager
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.records._
@@ -32,49 +35,57 @@ 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 org.apache.spark.{SparkContext, Logging}
+
+import org.apache.spark.{SparkConf, SparkContext, Logging}
import org.apache.spark.util.Utils
-import scala.collection.JavaConversions._
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
+ sparkConf: SparkConf) extends Logging {
+
+ def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+ this(args, new Configuration(), sparkConf)
-class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) extends Logging {
+ def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
- def this(args: ApplicationMasterArguments) = this(args, new Configuration())
-
- private var rpc: YarnRPC = YarnRPC.create(conf)
- private var resourceManager: AMRMProtocol = null
- private var appAttemptId: ApplicationAttemptId = null
- private var userThread: Thread = null
+ private val rpc: YarnRPC = YarnRPC.create(conf)
+ private var resourceManager: AMRMProtocol = _
+ private var appAttemptId: ApplicationAttemptId = _
+ private var userThread: Thread = _
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 var yarnAllocator: YarnAllocationHandler = _
+ 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
- // default to numWorkers * 2, with minimum of 3
- private val maxNumWorkerFailures = System.getProperty("spark.yarn.max.worker.failures",
- math.max(args.numWorkers * 2, 3).toString()).toInt
+
+ // Default to numWorkers * 2, with minimum of 3
+ private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures",
+ math.max(args.numWorkers * 2, 3))
def run() {
- // setup the directories so things go to yarn approved directories rather
- // then user specified and /tmp
+ // 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
+ // set the web ui port to be ephemeral for yarn so we don't conflict with
+ // other spark processes running on the same box
+ System.setProperty("spark.ui.port", "0")
+
+ // 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()
// 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
+ // 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 ?
+ // Hence args.workerCores = numCore disabled above. Any better option?
// Compute number of threads for akka
//val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory()
@@ -89,24 +100,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
// }
//}
// org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
-
+
ApplicationMaster.register(this)
// Start the user's JAR
userThread = startUserClass()
-
+
// 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
+ // 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()
-
- // Wait for the user class to Finish
+
+ // Wait for the user class to Finish
userThread.join()
System.exit(0)
@@ -124,65 +133,46 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
if (localDirs.isEmpty()) {
throw new Exception("Yarn Local dirs can't be empty")
}
- return localDirs
+ localDirs
}
-
+
private def getApplicationAttemptId(): ApplicationAttemptId = {
val envs = System.getenv()
val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV)
val containerId = ConverterUtils.toContainerId(containerIdString)
val appAttemptId = containerId.getApplicationAttemptId()
logInfo("ApplicationAttemptId: " + appAttemptId)
- return appAttemptId
+ appAttemptId
}
-
+
private def registerWithResourceManager(): AMRMProtocol = {
val rmAddress = NetUtils.createSocketAddr(yarnConf.get(
YarnConfiguration.RM_SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS))
logInfo("Connecting to ResourceManager at " + rmAddress)
- return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
+ rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
}
-
+
private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
logInfo("Registering the ApplicationMaster")
val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest])
.asInstanceOf[RegisterApplicationMasterRequest]
appMasterRequest.setApplicationAttemptId(appAttemptId)
- // Setting this to master host,port - so that the ApplicationReport at client has some sensible info.
+ // Setting this to master host,port - so that the ApplicationReport at client has some
+ // sensible info.
// Users can then monitor stderr/stdout on that node if required.
appMasterRequest.setHost(Utils.localHostName())
appMasterRequest.setRpcPort(0)
appMasterRequest.setTrackingUrl(uiAddress)
- return resourceManager.registerApplicationMaster(appMasterRequest)
- }
-
- private def waitForSparkMaster() {
- logInfo("Waiting for spark driver to be reachable.")
- var driverUp = false
- var tries = 0
- 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 {
- val socket = new Socket(driverHost, driverPort.toInt)
- socket.close()
- logInfo("Driver now available: " + driverHost + ":" + driverPort)
- driverUp = true
- } catch {
- case e: Exception =>
- logWarning("Failed to connect to driver at " + driverHost + ":" + driverPort + ", retrying")
- Thread.sleep(100)
- tries = tries + 1
- }
- }
+ resourceManager.registerApplicationMaster(appMasterRequest)
}
- private def startUserClass(): Thread = {
+ private def startUserClass(): Thread = {
logInfo("Starting the user JAR in a separate Thread")
- val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader)
- .getMethod("main", classOf[Array[String]])
+ val mainMethod = Class.forName(
+ args.userClass,
+ false /* initialize */ ,
+ Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
val t = new Thread {
override def run() {
var successed = false
@@ -207,7 +197,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
}
}
t.start()
- return t
+ t
}
// this need to happen before allocateWorkers
@@ -218,7 +208,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
ApplicationMaster.sparkContextRef.synchronized {
var count = 0
val waitTime = 10000L
- val numTries = System.getProperty("spark.yarn.ApplicationMaster.waitTries", "10").toInt
+ val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10)
while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) {
logInfo("Waiting for spark context initialization ... " + count)
count = count + 1
@@ -229,13 +219,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
if (null != sparkContext) {
uiAddress = sparkContext.ui.appUIAddress
- this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager,
- appAttemptId, args, sparkContext.preferredNodeLocationData)
+ this.yarnAllocator = YarnAllocationHandler.newAllocator(
+ yarnConf,
+ resourceManager,
+ appAttemptId,
+ args,
+ sparkContext.preferredNodeLocationData,
+ sparkContext.getConf)
} else {
- logWarning("Unable to retrieve sparkContext inspite of waiting for " + count * waitTime +
- ", numTries = " + numTries)
- this.yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager,
- appAttemptId, args)
+ logWarning("Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".
+ format(count * waitTime, numTries))
+ this.yarnAllocator = YarnAllocationHandler.newAllocator(
+ yarnConf,
+ resourceManager,
+ appAttemptId,
+ args,
+ sparkContext.getConf)
}
}
} finally {
@@ -251,53 +250,57 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
// Wait until all containers have finished
// TODO: This is a bit ugly. Can we make it nicer?
// TODO: Handle container failure
- while(yarnAllocator.getNumWorkersRunning < args.numWorkers &&
- // If user thread exists, then quit !
- userThread.isAlive) {
- if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
- finishApplicationMaster(FinalApplicationStatus.FAILED,
- "max number of worker failures reached")
- }
- yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
- ApplicationMaster.incrementAllocatorLoop(1)
- Thread.sleep(100)
+
+ // Exists the loop if the user thread exits.
+ while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) {
+ if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+ finishApplicationMaster(FinalApplicationStatus.FAILED,
+ "max number of worker failures reached")
+ }
+ yarnAllocator.allocateContainers(
+ math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
+ ApplicationMaster.incrementAllocatorLoop(1)
+ Thread.sleep(100)
}
} finally {
- // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT :
- // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks
+ // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT,
+ // so that the loop in ApplicationMaster#sparkContextInitialized() breaks.
ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
}
logInfo("All workers have launched.")
- // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+ // Launch a progress reporter thread, else the app will get killed after expiration
+ // (def: 10mins) timeout.
+ // TODO(harvey): Verify the timeout
if (userThread.isAlive) {
- // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
-
+ // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
// we want to be reasonably responsive without causing too many requests to RM.
- val schedulerInterval =
- System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong
+ val schedulerInterval =
+ sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000)
// must be <= timeoutInterval / 2.
val interval = math.min(timeoutInterval / 2, schedulerInterval)
+
launchReporterThread(interval)
}
}
private def launchReporterThread(_sleepTime: Long): Thread = {
- val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+ val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
val t = new Thread {
override def run() {
while (userThread.isAlive) {
if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
- finishApplicationMaster(FinalApplicationStatus.FAILED,
+ finishApplicationMaster(FinalApplicationStatus.FAILED,
"max number of worker failures reached")
}
val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning
if (missingWorkerCount > 0) {
- logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
+ logInfo("Allocating %d containers to make up for (potentially) lost containers".
+ format(missingWorkerCount))
yarnAllocator.allocateContainers(missingWorkerCount)
}
else sendProgress()
@@ -305,16 +308,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
}
}
}
- // setting to daemon status, though this is usually not a good idea.
+ // Setting to daemon status, though this is usually not a good idea.
t.setDaemon(true)
t.start()
logInfo("Started progress reporter thread - sleep time : " + sleepTime)
- return t
+ t
}
private def sendProgress() {
logDebug("Sending progress")
- // simulated with an allocate request with no nodes requested ...
+ // Simulated with an allocate request with no nodes requested ...
yarnAllocator.allocateContainers(0)
}
@@ -323,18 +326,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
for (container <- containers) {
logInfo("Launching shell command on a new container."
+ ", containerId=" + container.getId()
- + ", containerNode=" + container.getNodeId().getHost()
+ + ", containerNode=" + container.getNodeId().getHost()
+ ":" + container.getNodeId().getPort()
+ ", containerNodeURI=" + container.getNodeHttpAddress()
+ ", containerState" + container.getState()
- + ", containerResourceMemory"
+ + ", containerResourceMemory"
+ container.getResource().getMemory())
}
}
*/
def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
-
synchronized {
if (isFinished) {
return
@@ -348,19 +350,18 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
finishReq.setAppAttemptId(appAttemptId)
finishReq.setFinishApplicationStatus(status)
finishReq.setDiagnostics(diagnostics)
- // set tracking url to empty since we don't have a history server
+ // Set tracking url to empty since we don't have a history server.
finishReq.setTrackingUrl("")
resourceManager.finishApplicationMaster(finishReq)
-
}
/**
- * clean up the staging directory.
+ * Clean up the staging directory.
*/
- private def cleanupStagingDir() {
+ private def cleanupStagingDir() {
var stagingDirPath: Path = null
try {
- val preserveFiles = System.getProperty("spark.yarn.preserve.staging.files", "false").toBoolean
+ val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean
if (!preserveFiles) {
stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
if (stagingDirPath == null) {
@@ -371,13 +372,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
fs.delete(stagingDirPath, true)
}
} catch {
- case e: IOException =>
- logError("Failed to cleanup staging dir " + stagingDirPath, e)
+ case ioe: IOException =>
+ logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
}
}
- // The shutdown hook that runs when a signal is received AND during normal
- // close of the JVM.
+ // 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() {
@@ -387,16 +387,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration) e
if (appMaster.isLastAMRetry) appMaster.cleanupStagingDir()
}
}
-
+
}
object ApplicationMaster {
- // number of times to wait for the allocator loop to complete.
- // each loop iteration waits for 100ms, so maximum of 3 seconds.
+ // Number of times to wait for the allocator loop to complete.
+ // Each loop iteration waits for 100ms, so maximum of 3 seconds.
// This is to ensure that we have reasonable number of containers before we start
- // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be optimal as more
- // containers are available. Might need to handle this better.
+ // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
+ // optimal as more containers are available. Might need to handle this better.
private val ALLOCATOR_LOOP_WAIT_COUNT = 30
+
def incrementAllocatorLoop(by: Int) {
val count = yarnAllocatorLoop.getAndAdd(by)
if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
@@ -413,7 +414,8 @@ object ApplicationMaster {
applicationMasters.add(master)
}
- val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null)
+ val sparkContextRef: AtomicReference[SparkContext] =
+ new AtomicReference[SparkContext](null /* initialValue */)
val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0)
def sparkContextInitialized(sc: SparkContext): Boolean = {
@@ -423,27 +425,30 @@ object ApplicationMaster {
sparkContextRef.notifyAll()
}
- // 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.
+ // 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 the Client from declaring failure even though we exited properly.
+ // Note that this will unfortunately not properly clean up the staging files because it gets
+ // called too late, after 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'
+ Runtime.getRuntime().addShutdownHook(new Thread with Logging {
+ // This is not only logs, but also ensures that log system is initialized for this instance
+ // when we are actually 'run'-ing.
logInfo("Adding shutdown hook for context " + sc)
- override def run() {
- logInfo("Invoking sc stop from shutdown hook")
- sc.stop()
- // best case ...
+
+ override def run() {
+ logInfo("Invoking sc stop from shutdown hook")
+ sc.stop()
+ // Best case ...
for (master <- applicationMasters) {
master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
}
- }
- } )
+ }
+ })
}
- // Wait for initialization to complete and atleast 'some' nodes can get allocated
+ // Wait for initialization to complete and atleast 'some' nodes can get allocated.
yarnAllocatorLoop.synchronized {
while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
yarnAllocatorLoop.wait(1000L)
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 94e353af2e..23781ea35c 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -20,43 +20,53 @@ package org.apache.spark.deploy.yarn
import java.net.{InetAddress, UnknownHostException, URI}
import java.nio.ByteBuffer
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
-import org.apache.hadoop.fs.permission.FsPermission
-import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.net.NetUtils
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.client.YarnClientImpl
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{Apps, Records}
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
-import scala.collection.JavaConversions._
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.util.Utils
+import org.apache.spark.deploy.SparkHadoopUtil
+
-import org.apache.spark.Logging
+class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
+ extends YarnClientImpl with Logging {
+
+ def this(args: ClientArguments, sparkConf: SparkConf) =
+ this(args, new Configuration(), sparkConf)
+
+ def this(args: ClientArguments) = this(args, new SparkConf())
-class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl with Logging {
-
- def this(args: ClientArguments) = this(new Configuration(), args)
-
var rpc: YarnRPC = YarnRPC.create(conf)
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
val credentials = UserGroupInformation.getCurrentUser().getCredentials()
private val SPARK_STAGING: String = ".sparkStaging"
private val distCacheMgr = new ClientDistributedCacheManager()
- // staging directory is private! -> rwx--------
+ // Staging directory is private! -> rwx--------
val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short)
- // app files are world-wide readable and owner writable -> rw-r--r--
- val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short)
- def run() {
+ // App files are world-wide readable and owner writable -> rw-r--r--
+ val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short)
+
+ // for client user who want to monitor app status by itself.
+ def runApp() = {
validateArgs()
init(yarnConf)
@@ -78,21 +88,26 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
submitApp(appContext)
-
+ appId
+ }
+
+ def run() {
+ val appId = runApp()
monitorApplication(appId)
System.exit(0)
}
def validateArgs() = {
- Map((System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
+ Map(
+ (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
(args.userJar == null) -> "Error: You must specify a user jar!",
(args.userClass == null) -> "Error: You must specify a user class!",
- (args.numWorkers <= 0) -> "Error: You must specify atleast 1 worker!",
- (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) ->
- ("Error: AM memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD),
- (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) ->
- ("Error: Worker memory size must be greater then: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString()))
- .foreach { case(cond, errStr) =>
+ (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!",
+ (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be " +
+ "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
+ (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " +
+ "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD)
+ ).foreach { case(cond, errStr) =>
if (cond) {
logError(errStr)
args.printUsageAndExit(1)
@@ -106,33 +121,38 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
def logClusterResourceDetails() {
val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
- logInfo("Got Cluster metric info from ASM, numNodeManagers=" + clusterMetrics.getNumNodeManagers)
+ logInfo("Got Cluster metric info from ASM, numNodeManagers = " +
+ clusterMetrics.getNumNodeManagers)
val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
- logInfo("Queue info .. queueName=" + queueInfo.getQueueName + ", queueCurrentCapacity=" + queueInfo.getCurrentCapacity +
- ", queueMaxCapacity=" + queueInfo.getMaximumCapacity + ", queueApplicationCount=" + queueInfo.getApplications.size +
- ", queueChildQueueCount=" + queueInfo.getChildQueues.size)
+ logInfo( """Queue info ... queueName = %s, queueCurrentCapacity = %s, queueMaxCapacity = %s,
+ queueApplicationCount = %s, queueChildQueueCount = %s""".format(
+ queueInfo.getQueueName,
+ queueInfo.getCurrentCapacity,
+ queueInfo.getMaximumCapacity,
+ queueInfo.getApplications.size,
+ queueInfo.getChildQueues.size))
}
-
- def verifyClusterResources(app: GetNewApplicationResponse) = {
+
+ def verifyClusterResources(app: GetNewApplicationResponse) = {
val maxMem = app.getMaximumResourceCapability().getMemory()
logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
-
- // if we have requested more then the clusters max for a single resource then exit.
+
+ // If we have requested more then the clusters max for a single resource then exit.
if (args.workerMemory > maxMem) {
logError("the worker size is to large to run on this cluster " + args.workerMemory)
System.exit(1)
}
val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
if (amMem > maxMem) {
- logError("AM size is to large to run on this cluster " + amMem)
+ logError("AM size is to large to run on this cluster " + amMem)
System.exit(1)
}
- // We could add checks to make sure the entire cluster has enough resources but that involves getting
- // all the node reports and computing ourselves
+ // We could add checks to make sure the entire cluster has enough resources but that involves
+ // getting all the node reports and computing ourselves
}
-
+
def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
logInfo("Setting up application submission context for ASM")
val appContext = Records.newRecord(classOf[ApplicationSubmissionContext])
@@ -141,9 +161,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
return appContext
}
- /*
- * see if two file systems are the same or not.
- */
+ /** See if two file systems are the same or not. */
private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
val srcUri = srcFs.getUri()
val dstUri = destFs.getUri()
@@ -178,9 +196,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
return true
}
- /**
- * Copy the file into HDFS if needed.
- */
+ /** Copy the file into HDFS if needed. */
private def copyRemoteFile(
dstDir: Path,
originalPath: Path,
@@ -195,10 +211,9 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
fs.setReplication(newPath, replication)
if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
- }
- // resolve any symlinks in the URI path so using a "current" symlink
- // to point to a specific version shows the specific version
- // in the distributed cache configuration
+ }
+ // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+ // version shows the specific version in the distributed cache configuration
val qualPath = fs.makeQualified(newPath)
val fc = FileContext.getFileContext(qualPath.toUri(), conf)
val destPath = fc.resolvePath(qualPath)
@@ -207,8 +222,8 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
logInfo("Preparing Local resources")
- // Upload Spark and the application JAR to the remote file system if necessary
- // Add them as local resources to the AM
+ // Upload Spark and the application JAR to the remote file system if necessary. Add them as
+ // local resources to the AM.
val fs = FileSystem.get(conf)
val delegTokenRenewer = Master.getMasterPrincipal(conf)
@@ -219,7 +234,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
}
}
val dst = new Path(fs.getHomeDirectory(), appStagingDir)
- val replication = System.getProperty("spark.yarn.submit.file.replication", "3").toShort
+ val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort
if (UserGroupInformation.isSecurityEnabled()) {
val dstFs = dst.getFileSystem(conf)
@@ -230,7 +245,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
- Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
+ Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF"))
.foreach { case(destName, _localPath) =>
val localPath: String = if (_localPath != null) _localPath.trim() else ""
@@ -238,11 +253,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
var localURI = new URI(localPath)
// if not specified assume these are in the local filesystem to keep behavior like Hadoop
if (localURI.getScheme() == null) {
- localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString())
+ localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
}
val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
destName, statCache)
}
}
@@ -254,7 +269,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
val localPath = new Path(localURI)
val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
linkname, statCache, true)
}
}
@@ -266,7 +281,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
val localPath = new Path(localURI)
val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
linkname, statCache)
}
}
@@ -278,7 +293,7 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
val localPath = new Path(localURI)
val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
linkname, statCache)
}
}
@@ -286,45 +301,45 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
UserGroupInformation.getCurrentUser().addCredentials(credentials)
return localResources
}
-
+
def setupLaunchEnv(
- localResources: HashMap[String, LocalResource],
+ localResources: HashMap[String, LocalResource],
stagingDir: String): HashMap[String, String] = {
logInfo("Setting up the launch environment")
val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
val env = new HashMap[String, String]()
- Client.populateClasspath(yarnConf, log4jConfLocalRes != null, env)
+ Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
env("SPARK_YARN_MODE") = "true"
env("SPARK_YARN_STAGING_DIR") = stagingDir
- // set the environment variables to be passed on to the Workers
+ // Set the environment variables to be passed on to the Workers.
distCacheMgr.setDistFilesEnv(env)
distCacheMgr.setDistArchivesEnv(env)
- // allow users to specify some environment variables
+ // Allow users to specify some environment variables.
Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
- // Add each SPARK-* key to the environment
+ // Add each SPARK-* key to the environment.
System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
- return env
+ env
}
def userArgsToString(clientArgs: ClientArguments): String = {
val prefix = " --args "
val args = clientArgs.userArgs
val retval = new StringBuilder()
- for (arg <- args){
+ for (arg <- args) {
retval.append(prefix).append(" '").append(arg).append("' ")
}
-
retval.toString
}
- def createContainerLaunchContext(newApp: GetNewApplicationResponse,
- localResources: HashMap[String, LocalResource],
- env: HashMap[String, String]): ContainerLaunchContext = {
+ def createContainerLaunchContext(
+ newApp: GetNewApplicationResponse,
+ localResources: HashMap[String, LocalResource],
+ env: HashMap[String, String]): ContainerLaunchContext = {
logInfo("Setting up container launch context")
val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
amContainer.setLocalResources(localResources)
@@ -332,8 +347,10 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
+ // TODO(harvey): This can probably be a val.
var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
- (if (0 != (args.amMemory % minResMemory)) minResMemory else 0) - YarnAllocationHandler.MEMORY_OVERHEAD
+ ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
+ YarnAllocationHandler.MEMORY_OVERHEAD)
// Extra options for the JVM
var JAVA_OPTS = ""
@@ -341,16 +358,21 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
// Add Xmx for am memory
JAVA_OPTS += "-Xmx" + amMemory + "m "
- JAVA_OPTS += " -Djava.io.tmpdir=" +
+ 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
- // node, spark gc effects all other containers performance (which can also be other spark containers)
- // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is
- // limited to subset of cores on a node.
- if (env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))) {
- // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tenant machines
+ // 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 node,
+ // spark gc effects all other containers performance (which can also be other spark containers)
+ // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+ // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+ // of cores on a node.
+ val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
+ java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
+ if (useConcurrentAndIncrementalGC) {
+ // In our expts, using (default) throughput collector has severe perf ramnifications in
+ // multi-tenant machines
JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
JAVA_OPTS += " -XX:+CMSIncrementalMode "
JAVA_OPTS += " -XX:+CMSIncrementalPacing "
@@ -369,11 +391,11 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
}
- val commands = List[String](javaCommand +
+ val commands = List[String](javaCommand +
" -server " +
JAVA_OPTS +
- " org.apache.spark.deploy.yarn.ApplicationMaster" +
- " --class " + args.userClass +
+ " " + args.amClass +
+ " --class " + args.userClass +
" --jar " + args.userJar +
userArgsToString(args) +
" --worker-memory " + args.workerMemory +
@@ -383,29 +405,31 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
" 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
logInfo("Command for the ApplicationMaster: " + commands(0))
amContainer.setCommands(commands)
-
+
val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
- // Memory for the ApplicationMaster
+ // Memory for the ApplicationMaster.
capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
amContainer.setResource(capability)
- // Setup security tokens
+ // Setup security tokens.
val dob = new DataOutputBuffer()
credentials.writeTokenStorageToStream(dob)
amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData()))
- return amContainer
+ amContainer
}
-
+
def submitApp(appContext: ApplicationSubmissionContext) = {
- // Submit the application to the applications manager
+ // Submit the application to the applications manager.
logInfo("Submitting application to ASM")
super.submitApplication(appContext)
}
-
- def monitorApplication(appId: ApplicationId): Boolean = {
- while(true) {
- Thread.sleep(1000)
+
+ def monitorApplication(appId: ApplicationId): Boolean = {
+ val interval = sparkConf.getLong("spark.yarn.report.interval", 1000)
+
+ while (true) {
+ Thread.sleep(interval)
val report = super.getApplicationReport(appId)
logInfo("Application report from ASM: \n" +
@@ -422,16 +446,16 @@ class Client(conf: Configuration, args: ClientArguments) extends YarnClientImpl
"\t appTrackingUrl: " + report.getTrackingUrl() + "\n" +
"\t appUser: " + report.getUser()
)
-
+
val state = report.getYarnApplicationState()
val dsStatus = report.getFinalApplicationStatus()
- if (state == YarnApplicationState.FINISHED ||
+ if (state == YarnApplicationState.FINISHED ||
state == YarnApplicationState.FAILED ||
state == YarnApplicationState.KILLED) {
- return true
+ return true
}
}
- return true
+ true
}
}
@@ -445,9 +469,10 @@ object Client {
// Note that anything with SPARK prefix gets propagated to all (remote) processes
System.setProperty("SPARK_YARN_MODE", "true")
- val args = new ClientArguments(argStrings)
+ val sparkConf = new SparkConf
+ val args = new ClientArguments(argStrings, sparkConf)
- new Client(args).run
+ new Client(args, sparkConf).run
}
// Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
@@ -457,29 +482,28 @@ object Client {
}
}
- def populateClasspath(conf: Configuration, addLog4j: Boolean, env: HashMap[String, String]) {
+ def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
// If log4j present, ensure ours overrides all others
if (addLog4j) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
Path.SEPARATOR + LOG4J_PROP)
}
- // normally the users app.jar is last in case conflicts with spark jars
- val userClasspathFirst = System.getProperty("spark.yarn.user.classpath.first", "false")
- .toBoolean
+ // Normally the users app.jar is last in case conflicts with spark jars
+ val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean
if (userClasspathFirst) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
Path.SEPARATOR + APP_JAR)
}
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
Path.SEPARATOR + SPARK_JAR)
Client.populateHadoopClasspath(conf, env)
if (!userClasspathFirst) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
Path.SEPARATOR + APP_JAR)
}
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
Path.SEPARATOR + "*")
}
}
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
new file mode 100644
index 0000000000..ddfec1a4ac
--- /dev/null
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.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.deploy.yarn
+
+import java.net.Socket
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.net.NetUtils
+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 akka.actor._
+import akka.remote._
+import akka.actor.Terminated
+import org.apache.spark.{SparkConf, SparkContext, Logging}
+import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.scheduler.SplitInfo
+
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf)
+ extends Logging {
+
+ def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf)
+
+ def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
+
+ private val rpc: YarnRPC = YarnRPC.create(conf)
+ private var resourceManager: AMRMProtocol = _
+ private var appAttemptId: ApplicationAttemptId = _
+ private var reporterThread: Thread = _
+ private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+ private var yarnAllocator: YarnAllocationHandler = _
+ private var driverClosed:Boolean = false
+
+ val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0,
+ conf = sparkConf)._1
+ var actor: ActorRef = _
+
+ // This actor just working as a monitor to watch on Driver Actor.
+ class MonitorActor(driverUrl: String) extends Actor {
+
+ var driver: ActorSelection = _
+
+ override def preStart() {
+ logInfo("Listen to driver: " + driverUrl)
+ driver = context.actorSelection(driverUrl)
+ // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events.
+ driver ! "Hello"
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+ }
+
+ override def receive = {
+ case x: DisassociatedEvent =>
+ logInfo(s"Driver terminated or disconnected! Shutting down. $x")
+ driverClosed = true
+ }
+ }
+
+ def run() {
+
+ appAttemptId = getApplicationAttemptId()
+ resourceManager = registerWithResourceManager()
+ val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster()
+
+ // 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) {
+ // 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
+ }
+ }
+
+ waitForSparkMaster()
+
+ // Allocate all containers
+ allocateWorkers()
+
+ // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+ // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
+
+ val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+ // must be <= timeoutInterval/ 2.
+ // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM.
+ // so atleast 1 minute or timeoutInterval / 10 - whichever is higher.
+ val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval/ 10, 60000L))
+ reporterThread = launchReporterThread(interval)
+
+ // Wait for the reporter thread to Finish.
+ reporterThread.join()
+
+ finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+ actorSystem.shutdown()
+
+ logInfo("Exited")
+ System.exit(0)
+ }
+
+ private def getApplicationAttemptId(): ApplicationAttemptId = {
+ val envs = System.getenv()
+ val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV)
+ val containerId = ConverterUtils.toContainerId(containerIdString)
+ val appAttemptId = containerId.getApplicationAttemptId()
+ logInfo("ApplicationAttemptId: " + appAttemptId)
+ return appAttemptId
+ }
+
+ private def registerWithResourceManager(): AMRMProtocol = {
+ val rmAddress = NetUtils.createSocketAddr(yarnConf.get(
+ YarnConfiguration.RM_SCHEDULER_ADDRESS,
+ YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS))
+ logInfo("Connecting to ResourceManager at " + rmAddress)
+ return rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol]
+ }
+
+ private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+ logInfo("Registering the ApplicationMaster")
+ val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest])
+ .asInstanceOf[RegisterApplicationMasterRequest]
+ appMasterRequest.setApplicationAttemptId(appAttemptId)
+ // Setting this to master host,port - so that the ApplicationReport at client has some sensible info.
+ // 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("")
+ return resourceManager.registerApplicationMaster(appMasterRequest)
+ }
+
+ private def waitForSparkMaster() {
+ logInfo("Waiting for spark driver to be reachable.")
+ var driverUp = false
+ val hostport = args.userArgs(0)
+ val (driverHost, driverPort) = Utils.parseHostPort(hostport)
+ while(!driverUp) {
+ try {
+ val socket = new Socket(driverHost, driverPort)
+ socket.close()
+ logInfo("Master now available: " + driverHost + ":" + driverPort)
+ driverUp = true
+ } catch {
+ case e: Exception =>
+ logError("Failed to connect to driver at " + driverHost + ":" + driverPort)
+ Thread.sleep(100)
+ }
+ }
+ sparkConf.set("spark.driver.host", driverHost)
+ sparkConf.set("spark.driver.port", driverPort.toString)
+
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+ driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+ actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM")
+ }
+
+
+ private def allocateWorkers() {
+
+ // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now.
+ val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
+ scala.collection.immutable.Map()
+
+ yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId,
+ args, preferredNodeLocationData, sparkConf)
+
+ logInfo("Allocating " + args.numWorkers + " workers.")
+ // Wait until all containers have finished
+ // TODO: This is a bit ugly. Can we make it nicer?
+ // TODO: Handle container failure
+ while(yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+ yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0))
+ Thread.sleep(100)
+ }
+
+ logInfo("All workers have launched.")
+
+ }
+
+ // TODO: We might want to extend this to allocate more containers in case they die !
+ private def launchReporterThread(_sleepTime: Long): Thread = {
+ val sleepTime = if (_sleepTime <= 0 ) 0 else _sleepTime
+
+ val t = new Thread {
+ override def run() {
+ while (!driverClosed) {
+ val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning
+ if (missingWorkerCount > 0) {
+ logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers")
+ yarnAllocator.allocateContainers(missingWorkerCount)
+ }
+ else sendProgress()
+ Thread.sleep(sleepTime)
+ }
+ }
+ }
+ // setting to daemon status, though this is usually not a good idea.
+ t.setDaemon(true)
+ t.start()
+ logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+ return t
+ }
+
+ private def sendProgress() {
+ logDebug("Sending progress")
+ // simulated with an allocate request with no nodes requested ...
+ yarnAllocator.allocateContainers(0)
+ }
+
+ def finishApplicationMaster(status: FinalApplicationStatus) {
+
+ logInfo("finish ApplicationMaster with " + status)
+ val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest])
+ .asInstanceOf[FinishApplicationMasterRequest]
+ finishReq.setAppAttemptId(appAttemptId)
+ finishReq.setFinishApplicationStatus(status)
+ resourceManager.finishApplicationMaster(finishReq)
+ }
+
+}
+
+
+object WorkerLauncher {
+ def main(argStrings: Array[String]) {
+ val args = new ApplicationMasterArguments(argStrings)
+ new WorkerLauncher(args).run()
+ }
+}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index a4d6e1d87d..132630e5ef 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -21,52 +21,60 @@ import java.net.URI
import java.nio.ByteBuffer
import java.security.PrivilegedExceptionAction
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.DataOutputBuffer
import org.apache.hadoop.net.NetUtils
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
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.{Apps, ConverterUtils, Records, ProtoUtils}
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
-import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
+import org.apache.spark.{SparkConf, Logging}
+
-import org.apache.spark.Logging
+class WorkerRunnable(
+ container: Container,
+ conf: Configuration,
+ sparkConf: SparkConf,
+ masterAddress: String,
+ slaveId: String,
+ hostname: String,
+ workerMemory: Int,
+ workerCores: Int)
+ extends Runnable with Logging {
-class WorkerRunnable(container: Container, conf: Configuration, masterAddress: String,
- slaveId: String, hostname: String, workerMemory: Int, workerCores: Int)
- extends Runnable with Logging {
-
var rpc: YarnRPC = YarnRPC.create(conf)
- var cm: ContainerManager = null
+ var cm: ContainerManager = _
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
-
+
def run = {
logInfo("Starting Worker Container")
cm = connectToCM
startContainer
}
-
+
def startContainer = {
logInfo("Setting up ContainerLaunchContext")
-
+
val ctx = Records.newRecord(classOf[ContainerLaunchContext])
.asInstanceOf[ContainerLaunchContext]
-
+
ctx.setContainerId(container.getId())
ctx.setResource(container.getResource())
val localResources = prepareLocalResources
ctx.setLocalResources(localResources)
-
+
val env = prepareEnvironment
ctx.setEnvironment(env)
-
+
// Extra options for the JVM
var JAVA_OPTS = ""
// Set the JVM memory
@@ -79,17 +87,21 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
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
- // node, spark gc effects all other containers performance (which can also be other spark containers)
- // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in multi-tenant environments. Not sure how default java gc behaves if it is
- // limited to subset of cores on a node.
+ // 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 node, spark gc effects all other containers
+ // performance (which can also be other spark containers)
+ // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+ // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+ // of cores on a node.
/*
else {
// If no java_opts specified, default to using -XX:+CMSIncrementalMode
- // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont want to mess with it.
- // In our expts, using (default) throughput collector has severe perf ramnifications in multi-tennent machines
+ // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
+ // want to mess with it.
+ // In our expts, using (default) throughput collector has severe perf ramnifications in
+ // multi-tennent machines
// The options are based on
// http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
@@ -116,8 +128,10 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
val commands = List[String](javaCommand +
" -server " +
// Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
- // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in an inconsistent state.
- // TODO: If the OOM is not recoverable by rescheduling it on different node, then do 'something' to fail job ... akin to blacklisting trackers in mapred ?
+ // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
+ // an inconsistent state.
+ // TODO: If the OOM is not recoverable by rescheduling it on different node, then do
+ // 'something' to fail job ... akin to blacklisting trackers in mapred ?
" -XX:OnOutOfMemoryError='kill %p' " +
JAVA_OPTS +
" org.apache.spark.executor.CoarseGrainedExecutorBackend " +
@@ -129,7 +143,7 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
" 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
logInfo("Setting up worker with commands: " + commands)
ctx.setCommands(commands)
-
+
// Send the start request to the ContainerManager
val startReq = Records.newRecord(classOf[StartContainerRequest])
.asInstanceOf[StartContainerRequest]
@@ -137,7 +151,8 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
cm.startContainer(startReq)
}
- private def setupDistributedCache(file: String,
+ private def setupDistributedCache(
+ file: String,
rtype: LocalResourceType,
localResources: HashMap[String, LocalResource],
timestamp: String,
@@ -152,12 +167,11 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
amJarRsrc.setSize(size.toLong)
localResources(uri.getFragment()) = amJarRsrc
}
-
-
+
def prepareLocalResources: HashMap[String, LocalResource] = {
logInfo("Preparing Local resources")
val localResources = HashMap[String, LocalResource]()
-
+
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(',')
@@ -179,30 +193,30 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
timeStamps(i), fileSizes(i), visibilities(i))
}
}
-
+
logInfo("Prepared Local resources " + localResources)
return localResources
}
-
+
def prepareEnvironment: HashMap[String, String] = {
val env = new HashMap[String, String]()
- Client.populateClasspath(yarnConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+ Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
- // allow users to specify some environment variables
+ // Allow users to specify some environment variables
Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
return env
}
-
+
def connectToCM: ContainerManager = {
val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort()
val cmAddress = NetUtils.createSocketAddr(cmHostPortStr)
logInfo("Connecting to ContainerManager at " + cmHostPortStr)
- // use doAs and remoteUser here so we can add the container token and not
- // pollute the current users credentials with all of the individual container tokens
+ // Use doAs and remoteUser here so we can add the container token and not pollute the current
+ // users credentials with all of the individual container tokens
val user = UserGroupInformation.createRemoteUser(container.getId().toString())
val containerToken = container.getContainerToken()
if (containerToken != null) {
@@ -218,5 +232,5 @@ class WorkerRunnable(container: Container, conf: Configuration, masterAddress: S
})
proxy
}
-
+
}
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
index 507a0743fd..e91257be8e 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -17,55 +17,71 @@
package org.apache.spark.deploy.yarn
-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}
-import org.apache.spark.scheduler.cluster.{ClusterScheduler, CoarseGrainedSchedulerBackend}
-import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
-import org.apache.hadoop.yarn.util.{RackResolver, Records}
+import java.lang.{Boolean => JBoolean}
+import java.util.{Collections, Set => JSet}
import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
import java.util.concurrent.atomic.AtomicInteger
-import org.apache.hadoop.yarn.api.AMRMProtocol
-import collection.JavaConversions._
-import collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
+import scala.collection
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.Utils
+
import org.apache.hadoop.conf.Configuration
-import java.util.{Collections, Set => JSet}
-import java.lang.{Boolean => JBoolean}
+import org.apache.hadoop.yarn.api.AMRMProtocol
+import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId}
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus}
+import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest}
+import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
+import org.apache.hadoop.yarn.util.{RackResolver, Records}
-object AllocationType extends Enumeration ("HOST", "RACK", "ANY") {
+
+object AllocationType extends Enumeration {
type AllocationType = Value
val HOST, RACK, ANY = Value
}
-// too many params ? refactor it 'somehow' ?
-// needs to be mt-safe
-// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive : should make it
-// more proactive and decoupled.
+// TODO:
+// Too many params.
+// Needs to be mt-safe
+// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should
+// make it more proactive and decoupled.
+
// Note that right now, we assume all node asks as uniform in terms of capabilities and priority
-// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for more info
-// on how we are requesting for containers.
-private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceManager: AMRMProtocol,
- val appAttemptId: ApplicationAttemptId,
- val maxWorkers: Int, val workerMemory: Int, val workerCores: Int,
- val preferredHostToCount: Map[String, Int],
- val preferredRackToCount: Map[String, Int])
+// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for
+// more info on how we are requesting for containers.
+private[yarn] class YarnAllocationHandler(
+ val conf: Configuration,
+ val resourceManager: AMRMProtocol,
+ val appAttemptId: ApplicationAttemptId,
+ val maxWorkers: Int,
+ val workerMemory: Int,
+ val workerCores: Int,
+ val preferredHostToCount: Map[String, Int],
+ val preferredRackToCount: Map[String, Int],
+ val sparkConf: SparkConf)
extends Logging {
-
-
// These three are locked on allocatedHostToContainersMap. Complementary data structures
// allocatedHostToContainersMap : containers which are running : host, Set<containerid>
- // allocatedContainerToHostMap: container to host mapping
- private val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]()
+ // allocatedContainerToHostMap: container to host mapping.
+ private val allocatedHostToContainersMap =
+ new HashMap[String, collection.mutable.Set[ContainerId]]()
+
private val allocatedContainerToHostMap = new HashMap[ContainerId, String]()
- // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an allocated node)
- // As with the two data structures above, tightly coupled with them, and to be locked on allocatedHostToContainersMap
+
+ // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an
+ // allocated node)
+ // As with the two data structures above, tightly coupled with them, and to be locked on
+ // allocatedHostToContainersMap
private val allocatedRackCount = new HashMap[String, Int]()
- // containers which have been released.
+ // Containers which have been released.
private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]()
- // containers to be released in next request to RM
+ // Containers to be released in next request to RM
private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean]
private val numWorkersRunning = new AtomicInteger()
@@ -83,23 +99,31 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
}
def allocateContainers(workersToRequest: Int) {
- // We need to send the request only once from what I understand ... but for now, not modifying this much.
+ // We need to send the request only once from what I understand ... but for now, not modifying
+ // this much.
// Keep polling the Resource Manager for containers
val amResp = allocateWorkerResources(workersToRequest).getAMResponse
val _allocatedContainers = amResp.getAllocatedContainers()
- if (_allocatedContainers.size > 0) {
-
- logDebug("Allocated " + _allocatedContainers.size + " containers, current count " +
- numWorkersRunning.get() + ", to-be-released " + releasedContainerList +
- ", pendingReleaseContainers : " + pendingReleaseContainers)
- logDebug("Cluster Resources: " + amResp.getAvailableResources)
+ if (_allocatedContainers.size > 0) {
+ logDebug("""
+ Allocated containers: %d
+ Current worker count: %d
+ Containers released: %s
+ Containers to be released: %s
+ Cluster resources: %s
+ """.format(
+ _allocatedContainers.size,
+ numWorkersRunning.get(),
+ releasedContainerList,
+ pendingReleaseContainers,
+ amResp.getAvailableResources))
val hostToContainers = new HashMap[String, ArrayBuffer[Container]]()
- // ignore if not satisfying constraints {
+ // Ignore if not satisfying constraints {
for (container <- _allocatedContainers) {
if (isResourceConstraintSatisfied(container)) {
// allocatedContainers += container
@@ -113,8 +137,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
else releasedContainerList.add(container.getId())
}
- // Find the appropriate containers to use
- // Slightly non trivial groupBy I guess ...
+ // Find the appropriate containers to use. Slightly non trivial groupBy ...
val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
val offRackContainers = new HashMap[String, ArrayBuffer[Container]]()
@@ -134,21 +157,22 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
remainingContainers = null
}
else if (requiredHostCount > 0) {
- // container list has more containers than we need for data locality.
- // Split into two : data local container count of (remainingContainers.size - requiredHostCount)
- // and rest as remainingContainer
- val (dataLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredHostCount)
+ // Container list has more containers than we need for data locality.
+ // Split into two : data local container count of (remainingContainers.size -
+ // requiredHostCount) and rest as remainingContainer
+ val (dataLocal, remaining) = remainingContainers.splitAt(
+ remainingContainers.size - requiredHostCount)
dataLocalContainers.put(candidateHost, dataLocal)
// remainingContainers = remaining
// yarn has nasty habit of allocating a tonne of containers on a host - discourage this :
- // add remaining to release list. If we have insufficient containers, next allocation cycle
- // will reallocate (but wont treat it as data local)
+ // add remaining to release list. If we have insufficient containers, next allocation
+ // cycle will reallocate (but wont treat it as data local)
for (container <- remaining) releasedContainerList.add(container.getId())
remainingContainers = null
}
- // now rack local
+ // Now rack local
if (remainingContainers != null){
val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
@@ -161,15 +185,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
if (requiredRackCount >= remainingContainers.size){
// Add all to dataLocalContainers
dataLocalContainers.put(rack, remainingContainers)
- // all consumed
+ // All consumed
remainingContainers = null
}
else if (requiredRackCount > 0) {
// container list has more containers than we need for data locality.
- // Split into two : data local container count of (remainingContainers.size - requiredRackCount)
- // and rest as remainingContainer
- val (rackLocal, remaining) = remainingContainers.splitAt(remainingContainers.size - requiredRackCount)
- val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, new ArrayBuffer[Container]())
+ // Split into two : data local container count of (remainingContainers.size -
+ // requiredRackCount) and rest as remainingContainer
+ val (rackLocal, remaining) = remainingContainers.splitAt(
+ remainingContainers.size - requiredRackCount)
+ val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack,
+ new ArrayBuffer[Container]())
existingRackLocal ++= rackLocal
remainingContainers = remaining
@@ -185,13 +211,13 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
// Now that we have split the containers into various groups, go through them in order :
// first host local, then rack local and then off rack (everything else).
- // Note that the list we create below tries to ensure that not all containers end up within a host
- // if there are sufficiently large number of hosts/containers.
+ // Note that the list we create below tries to ensure that not all containers end up within a
+ // host if there are sufficiently large number of hosts/containers.
val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size)
- allocatedContainers ++= ClusterScheduler.prioritizeContainers(dataLocalContainers)
- allocatedContainers ++= ClusterScheduler.prioritizeContainers(rackLocalContainers)
- allocatedContainers ++= ClusterScheduler.prioritizeContainers(offRackContainers)
+ allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers)
+ allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers)
+ allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers)
// Run each of the allocated containers
for (container <- allocatedContainers) {
@@ -199,52 +225,64 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
val workerHostname = container.getNodeId.getHost
val containerId = container.getId
- assert (container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
+ assert(
+ container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
if (numWorkersRunningNow > maxWorkers) {
- logInfo("Ignoring container " + containerId + " at host " + workerHostname +
- " .. we already have required number of containers")
+ logInfo("""Ignoring container %s at host %s, since we already have the required number of
+ containers for it.""".format(containerId, workerHostname))
releasedContainerList.add(containerId)
// reset counter back to old value.
numWorkersRunning.decrementAndGet()
}
else {
- // deallocate + allocate can result in reusing id's wrongly - so use a different counter (workerIdCounter)
+ // Deallocate + allocate can result in reusing id's wrongly - so use a different counter
+ // (workerIdCounter)
val workerId = workerIdCounter.incrementAndGet().toString
- val driverUrl = "akka://spark@%s:%s/user/%s".format(
- System.getProperty("spark.driver.host"), System.getProperty("spark.driver.port"),
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+ sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"),
CoarseGrainedSchedulerBackend.ACTOR_NAME)
logInfo("launching container on " + containerId + " host " + workerHostname)
- // just to be safe, simply remove it from pendingReleaseContainers. Should not be there, but ..
+ // Just to be safe, simply remove it from pendingReleaseContainers.
+ // Should not be there, but ..
pendingReleaseContainers.remove(containerId)
val rack = YarnAllocationHandler.lookupRack(conf, workerHostname)
allocatedHostToContainersMap.synchronized {
- val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, new HashSet[ContainerId]())
+ val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname,
+ new HashSet[ContainerId]())
containerSet += containerId
allocatedContainerToHostMap.put(containerId, workerHostname)
- if (rack != null) allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+ if (rack != null) {
+ allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+ }
}
new Thread(
- new WorkerRunnable(container, conf, driverUrl, workerId,
+ new WorkerRunnable(container, conf, sparkConf, driverUrl, workerId,
workerHostname, workerMemory, workerCores)
).start()
}
}
- logDebug("After allocated " + allocatedContainers.size + " containers (orig : " +
- _allocatedContainers.size + "), current count " + numWorkersRunning.get() +
- ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
+ logDebug("""
+ Finished processing %d containers.
+ Current number of workers running: %d,
+ releasedContainerList: %s,
+ pendingReleaseContainers: %s
+ """.format(
+ allocatedContainers.size,
+ numWorkersRunning.get(),
+ releasedContainerList,
+ pendingReleaseContainers))
}
val completedContainers = amResp.getCompletedContainersStatuses()
if (completedContainers.size > 0){
- logDebug("Completed " + completedContainers.size + " containers, current count " + numWorkersRunning.get() +
- ", to-be-released " + releasedContainerList + ", pendingReleaseContainers : " + pendingReleaseContainers)
-
+ logDebug("Completed %d containers, to-be-released: %s".format(
+ completedContainers.size, releasedContainerList))
for (completedContainer <- completedContainers){
val containerId = completedContainer.getContainerId
@@ -253,16 +291,17 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
pendingReleaseContainers.remove(containerId)
}
else {
- // simply decrement count - next iteration of ReporterThread will take care of allocating !
+ // Simply decrement count - next iteration of ReporterThread will take care of allocating.
numWorkersRunning.decrementAndGet()
- logInfo("Container completed not by us ? nodeId: " + containerId + ", state " + completedContainer.getState +
- " httpaddress: " + completedContainer.getDiagnostics + " exit status: " + completedContainer.getExitStatus())
-
+ logInfo("Completed container %s (state: %s, exit status: %s)".format(
+ containerId,
+ completedContainer.getState,
+ completedContainer.getExitStatus()))
// Hadoop 2.2.X added a ContainerExitStatus we should switch to use
// there are some exit status' we shouldn't necessarily count against us, but for
// now I think its ok as none of the containers are expected to exit
if (completedContainer.getExitStatus() != 0) {
- logInfo("Container marked as failed: " + containerId)
+ logInfo("Container marked as failed: " + containerId)
numWorkersFailed.incrementAndGet()
}
}
@@ -281,7 +320,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
allocatedContainerToHostMap -= containerId
- // doing this within locked context, sigh ... move to outside ?
+ // Doing this within locked context, sigh ... move to outside ?
val rack = YarnAllocationHandler.lookupRack(conf, host)
if (rack != null) {
val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1
@@ -291,9 +330,16 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
}
}
}
- logDebug("After completed " + completedContainers.size + " containers, current count " +
- numWorkersRunning.get() + ", to-be-released " + releasedContainerList +
- ", pendingReleaseContainers : " + pendingReleaseContainers)
+ logDebug("""
+ Finished processing %d completed containers.
+ Current number of workers running: %d,
+ releasedContainerList: %s,
+ pendingReleaseContainers: %s
+ """.format(
+ completedContainers.size,
+ numWorkersRunning.get(),
+ releasedContainerList,
+ pendingReleaseContainers))
}
}
@@ -347,7 +393,7 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
// default.
if (numWorkers <= 0 || preferredHostToCount.isEmpty) {
- logDebug("numWorkers: " + numWorkers + ", host preferences ? " + preferredHostToCount.isEmpty)
+ logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty)
resourceRequests = List(
createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY))
}
@@ -360,17 +406,24 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost)
if (requiredCount > 0) {
- hostContainerRequests +=
- createResourceRequest(AllocationType.HOST, candidateHost, requiredCount, YarnAllocationHandler.PRIORITY)
+ hostContainerRequests += createResourceRequest(
+ AllocationType.HOST,
+ candidateHost,
+ requiredCount,
+ YarnAllocationHandler.PRIORITY)
}
}
- val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(hostContainerRequests.toList)
+ val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests(
+ hostContainerRequests.toList)
- val anyContainerRequests: ResourceRequest =
- createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)
+ val anyContainerRequests: ResourceRequest = createResourceRequest(
+ AllocationType.ANY,
+ resource = null,
+ numWorkers,
+ YarnAllocationHandler.PRIORITY)
- val containerRequests: ArrayBuffer[ResourceRequest] =
- new ArrayBuffer[ResourceRequest](hostContainerRequests.size() + rackContainerRequests.size() + 1)
+ val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](
+ hostContainerRequests.size + rackContainerRequests.size + 1)
containerRequests ++= hostContainerRequests
containerRequests ++= rackContainerRequests
@@ -389,52 +442,59 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
req.addAllReleases(releasedContainerList)
if (numWorkers > 0) {
- logInfo("Allocating " + numWorkers + " worker containers with " + (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + " of memory each.")
+ logInfo("Allocating %d worker containers with %d of memory each.".format(numWorkers,
+ workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))
}
else {
logDebug("Empty allocation req .. release : " + releasedContainerList)
}
- for (req <- resourceRequests) {
- logInfo("rsrcRequest ... host : " + req.getHostName + ", numContainers : " + req.getNumContainers +
- ", p = " + req.getPriority().getPriority + ", capability: " + req.getCapability)
+ for (request <- resourceRequests) {
+ logInfo("ResourceRequest (host : %s, num containers: %d, priority = %s , capability : %s)".
+ format(
+ request.getHostName,
+ request.getNumContainers,
+ request.getPriority,
+ request.getCapability))
}
resourceManager.allocate(req)
}
- private def createResourceRequest(requestType: AllocationType.AllocationType,
- resource:String, numWorkers: Int, priority: Int): ResourceRequest = {
+ private def createResourceRequest(
+ requestType: AllocationType.AllocationType,
+ resource:String,
+ numWorkers: Int,
+ priority: Int): ResourceRequest = {
// If hostname specified, we need atleast two requests - node local and rack local.
// There must be a third request - which is ANY : that will be specially handled.
requestType match {
case AllocationType.HOST => {
- assert (YarnAllocationHandler.ANY_HOST != resource)
-
+ assert(YarnAllocationHandler.ANY_HOST != resource)
val hostname = resource
val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority)
- // add to host->rack mapping
+ // Add to host->rack mapping
YarnAllocationHandler.populateRackInfo(conf, hostname)
nodeLocal
}
-
case AllocationType.RACK => {
val rack = resource
createResourceRequestImpl(rack, numWorkers, priority)
}
-
- case AllocationType.ANY => {
- createResourceRequestImpl(YarnAllocationHandler.ANY_HOST, numWorkers, priority)
- }
-
- case _ => throw new IllegalArgumentException("Unexpected/unsupported request type .. " + requestType)
+ case AllocationType.ANY => createResourceRequestImpl(
+ YarnAllocationHandler.ANY_HOST, numWorkers, priority)
+ case _ => throw new IllegalArgumentException(
+ "Unexpected/unsupported request type: " + requestType)
}
}
- private def createResourceRequestImpl(hostname:String, numWorkers: Int, priority: Int): ResourceRequest = {
+ private def createResourceRequestImpl(
+ hostname:String,
+ numWorkers: Int,
+ priority: Int): ResourceRequest = {
val rsrcRequest = Records.newRecord(classOf[ResourceRequest])
val memCapability = Records.newRecord(classOf[Resource])
@@ -455,11 +515,11 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
def createReleasedContainerList(): ArrayBuffer[ContainerId] = {
val retval = new ArrayBuffer[ContainerId](1)
- // iterator on COW list ...
+ // Iterator on COW list ...
for (container <- releasedContainerList.iterator()){
retval += container
}
- // remove from the original list.
+ // Remove from the original list.
if (! retval.isEmpty) {
releasedContainerList.removeAll(retval)
for (v <- retval) pendingReleaseContainers.put(v, true)
@@ -474,14 +534,14 @@ private[yarn] class YarnAllocationHandler(val conf: Configuration, val resourceM
object YarnAllocationHandler {
val ANY_HOST = "*"
- // all requests are issued with same priority : we do not (yet) have any distinction between
+ // All requests are issued with same priority : we do not (yet) have any distinction between
// request types (like map/reduce in hadoop for example)
val PRIORITY = 1
// Additional memory overhead - in mb
val MEMORY_OVERHEAD = 384
- // host to rack map - saved from allocation requests
+ // Host to rack map - saved from allocation requests
// We are expecting this not to change.
// Note that it is possible for this to change : and RM will indicate that to us via update
// response to allocate. But we are punting on handling that for now.
@@ -489,38 +549,75 @@ object YarnAllocationHandler {
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,
+ sparkConf: SparkConf): YarnAllocationHandler = {
+
+ new YarnAllocationHandler(
+ conf,
+ resourceManager,
+ appAttemptId,
+ args.numWorkers,
+ args.workerMemory,
+ args.workerCores,
+ Map[String, Int](),
+ Map[String, Int](),
+ sparkConf)
}
- def newAllocator(conf: Configuration,
- resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
- args: ApplicationMasterArguments,
- map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
+ def newAllocator(
+ conf: Configuration,
+ resourceManager: AMRMProtocol,
+ appAttemptId: ApplicationAttemptId,
+ args: ApplicationMasterArguments,
+ map: collection.Map[String,
+ collection.Set[SplitInfo]],
+ sparkConf: SparkConf): YarnAllocationHandler = {
val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
-
- new YarnAllocationHandler(conf, resourceManager, appAttemptId, args.numWorkers,
- args.workerMemory, args.workerCores, hostToCount, rackToCount)
+ new YarnAllocationHandler(
+ conf,
+ resourceManager,
+ appAttemptId,
+ args.numWorkers,
+ args.workerMemory,
+ args.workerCores,
+ hostToCount,
+ rackToCount,
+ sparkConf)
}
- def newAllocator(conf: Configuration,
- resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId,
- maxWorkers: Int, workerMemory: Int, workerCores: Int,
- map: collection.Map[String, collection.Set[SplitInfo]]): YarnAllocationHandler = {
+ def newAllocator(
+ conf: Configuration,
+ resourceManager: AMRMProtocol,
+ appAttemptId: ApplicationAttemptId,
+ maxWorkers: Int,
+ workerMemory: Int,
+ workerCores: Int,
+ map: collection.Map[String, collection.Set[SplitInfo]],
+ sparkConf: SparkConf): YarnAllocationHandler = {
val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
- new YarnAllocationHandler(conf, resourceManager, appAttemptId, maxWorkers,
- workerMemory, workerCores, hostToCount, rackToCount)
+ new YarnAllocationHandler(
+ conf,
+ resourceManager,
+ appAttemptId,
+ maxWorkers,
+ workerMemory,
+ workerCores,
+ hostToCount,
+ rackToCount,
+ sparkConf)
}
// A simple method to copy the split info map.
- private def generateNodeToWeight(conf: Configuration, input: collection.Map[String, collection.Set[SplitInfo]]) :
+ private def generateNodeToWeight(
+ conf: Configuration,
+ input: collection.Map[String, collection.Set[SplitInfo]]) :
// host to count, rack to count
(Map[String, Int], Map[String, Int]) = {
@@ -544,7 +641,7 @@ object YarnAllocationHandler {
}
def lookupRack(conf: Configuration, host: String): String = {
- if (! hostToRack.contains(host)) populateRackInfo(conf, host)
+ if (!hostToRack.contains(host)) populateRackInfo(conf, host)
hostToRack.get(host)
}
@@ -567,10 +664,12 @@ object YarnAllocationHandler {
val rack = rackInfo.getNetworkLocation
hostToRack.put(hostname, rack)
if (! rackToHostSet.containsKey(rack)) {
- rackToHostSet.putIfAbsent(rack, Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
+ rackToHostSet.putIfAbsent(rack,
+ Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
}
rackToHostSet.get(rack).add(hostname)
+ // TODO(harvey): Figure out this comment...
// Since RackResolver caches, we are disabling this for now ...
} /* else {
// right ? Else we will keep calling rack resolver in case we cant resolve rack info ...
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
index f76a5ddd39..f76a5ddd39 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index 852dbd7dab..1419f215c7 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -17,27 +17,33 @@
package org.apache.spark.deploy.yarn
-import org.apache.spark.util.MemoryParam
-import org.apache.spark.util.IntParam
-import collection.mutable.{ArrayBuffer, HashMap}
+import scala.collection.mutable.{ArrayBuffer, HashMap}
+
+import org.apache.spark.SparkConf
import org.apache.spark.scheduler.{InputFormatInfo, SplitInfo}
+import org.apache.spark.util.IntParam
+import org.apache.spark.util.MemoryParam
-// TODO: Add code and support for ensuring that yarn resource 'asks' are location aware !
-class ClientArguments(val args: Array[String]) {
+
+// TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware !
+class ClientArguments(val args: Array[String], val sparkConf: SparkConf) {
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]()
- var workerMemory = 1024
+ var workerMemory = 1024 // MB
var workerCores = 1
var numWorkers = 2
- var amQueue = System.getProperty("QUEUE", "default")
- var amMemory: Int = 512
+ var amQueue = sparkConf.get("QUEUE", "default")
+ var amMemory: Int = 512 // MB
+ var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster"
var appName: String = "Spark"
// TODO
var inputFormatInfo: List[InputFormatInfo] = null
+ // TODO(harvey)
+ var priority = 0
parseArgs(args.toList)
@@ -47,8 +53,7 @@ class ClientArguments(val args: Array[String]) {
var args = inputArgs
- while (! args.isEmpty) {
-
+ while (!args.isEmpty) {
args match {
case ("--jar") :: value :: tail =>
userJar = value
@@ -62,6 +67,10 @@ class ClientArguments(val args: Array[String]) {
userArgsBuffer += value
args = tail
+ case ("--master-class") :: value :: tail =>
+ amClass = value
+ args = tail
+
case ("--master-memory") :: MemoryParam(value) :: tail =>
amMemory = value
args = tail
@@ -84,6 +93,7 @@ class ClientArguments(val args: Array[String]) {
case ("--name") :: value :: tail =>
appName = value
+ args = tail
case ("--addJars") :: value :: tail =>
addJars = value
@@ -119,19 +129,20 @@ class ClientArguments(val args: Array[String]) {
System.err.println(
"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" +
- " --args ARGS Arguments to be passed to your application's main class.\n" +
- " Mutliple invocations are possible, each will be passed in order.\n" +
- " --num-workers NUM Number of workers to start (Default: 2)\n" +
- " --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" +
- " --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."
+ " --jar JAR_PATH Path to your application's JAR file (required)\n" +
+ " --class CLASS_NAME Name of your application's main class (required)\n" +
+ " --args ARGS Arguments to be passed to your application's main class.\n" +
+ " Mutliple invocations are possible, each will be passed in order.\n" +
+ " --num-workers NUM Number of workers to start (Default: 2)\n" +
+ " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" +
+ " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\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" +
+ " --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/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
index 5f159b073f..5f159b073f 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
index 2ba2366ead..2ba2366ead 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala
diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
new file mode 100644
index 0000000000..522e0a9ad7
--- /dev/null
+++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala
@@ -0,0 +1,48 @@
+/*
+ * 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 org.apache.spark._
+import org.apache.hadoop.conf.Configuration
+import org.apache.spark.deploy.yarn.YarnAllocationHandler
+import org.apache.spark.scheduler.TaskSchedulerImpl
+import org.apache.spark.util.Utils
+
+/**
+ *
+ * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM.
+ */
+private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) {
+
+ def this(sc: SparkContext) = this(sc, new Configuration())
+
+ // By default, rack is unknown
+ override def getRackForHost(hostPort: String): Option[String] = {
+ val host = Utils.parseHostPort(hostPort)._1
+ val retval = YarnAllocationHandler.lookupRack(conf, host)
+ if (retval != null) Some(retval) else None
+ }
+
+ override def postStartHook() {
+
+ // The yarn application is running, but the worker might not yet ready
+ // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt
+ Thread.sleep(2000L)
+ logInfo("YarnClientClusterScheduler.postStartHook done")
+ }
+}
diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
new file mode 100644
index 0000000000..4b1b5da048
--- /dev/null
+++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala
@@ -0,0 +1,112 @@
+/*
+ * 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 org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState}
+import org.apache.spark.{SparkException, Logging, SparkContext}
+import org.apache.spark.deploy.yarn.{Client, ClientArguments}
+import org.apache.spark.scheduler.TaskSchedulerImpl
+
+private[spark] class YarnClientSchedulerBackend(
+ scheduler: TaskSchedulerImpl,
+ sc: SparkContext)
+ extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem)
+ with Logging {
+
+ var client: Client = null
+ var appId: ApplicationId = null
+
+ override def start() {
+ super.start()
+
+ val defalutWorkerCores = "2"
+ val defalutWorkerMemory = "512m"
+ val defaultWorkerNumber = "1"
+
+ val userJar = System.getenv("SPARK_YARN_APP_JAR")
+ val distFiles = System.getenv("SPARK_YARN_DIST_FILES")
+ var workerCores = System.getenv("SPARK_WORKER_CORES")
+ var workerMemory = System.getenv("SPARK_WORKER_MEMORY")
+ var workerNumber = System.getenv("SPARK_WORKER_INSTANCES")
+
+ if (userJar == null)
+ throw new SparkException("env SPARK_YARN_APP_JAR is not set")
+
+ if (workerCores == null)
+ workerCores = defalutWorkerCores
+ if (workerMemory == null)
+ workerMemory = defalutWorkerMemory
+ if (workerNumber == null)
+ workerNumber = defaultWorkerNumber
+
+ val driverHost = conf.get("spark.driver.host")
+ val driverPort = conf.get("spark.driver.port")
+ val hostport = driverHost + ":" + driverPort
+
+ val argsArray = Array[String](
+ "--class", "notused",
+ "--jar", userJar,
+ "--args", hostport,
+ "--worker-memory", workerMemory,
+ "--worker-cores", workerCores,
+ "--num-workers", workerNumber,
+ "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher",
+ "--files", distFiles
+ )
+
+ val args = new ClientArguments(argsArray, conf)
+ client = new Client(args, conf)
+ appId = client.runApp()
+ waitForApp()
+ }
+
+ def waitForApp() {
+
+ // TODO : need a better way to find out whether the workers are ready or not
+ // maybe by resource usage report?
+ while(true) {
+ val report = client.getApplicationReport(appId)
+
+ logInfo("Application report from ASM: \n" +
+ "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
+ "\t appStartTime: " + report.getStartTime() + "\n" +
+ "\t yarnAppState: " + report.getYarnApplicationState() + "\n"
+ )
+
+ // Ready to go, or already gone.
+ val state = report.getYarnApplicationState()
+ if (state == YarnApplicationState.RUNNING) {
+ return
+ } else if (state == YarnApplicationState.FINISHED ||
+ state == YarnApplicationState.FAILED ||
+ state == YarnApplicationState.KILLED) {
+ throw new SparkException("Yarn application already ended," +
+ "might be killed or not able to launch application master.")
+ }
+
+ Thread.sleep(1000)
+ }
+ }
+
+ override def stop() {
+ super.stop()
+ client.stop()
+ logInfo("Stoped")
+ }
+
+}
diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
index 29b3f22e13..a4638cc863 100644
--- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala
+++ b/yarn/common/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.scheduler.TaskSchedulerImpl
import org.apache.spark.util.Utils
import org.apache.hadoop.conf.Configuration
@@ -26,7 +27,7 @@ import org.apache.hadoop.conf.Configuration
*
* This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of ApplicationMaster, etc is done
*/
-private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends ClusterScheduler(sc) {
+private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) {
logInfo("Created YarnClusterScheduler")
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
index 2941356bc5..2941356bc5 100644
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
+++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 8a065c6d7d..aea8b0cdde 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -25,15 +25,14 @@
</parent>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-yarn_2.9.3</artifactId>
- <packaging>jar</packaging>
- <name>Spark Project YARN Support</name>
- <url>http://spark.incubator.apache.org/</url>
-
+ <artifactId>yarn-parent_2.10</artifactId>
+ <packaging>pom</packaging>
+ <name>Spark Project YARN Parent POM</name>
+
<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
- <artifactId>spark-core_2.9.3</artifactId>
+ <artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
@@ -63,7 +62,7 @@
</dependency>
<dependency>
<groupId>org.scalatest</groupId>
- <artifactId>scalatest_2.9.3</artifactId>
+ <artifactId>scalatest_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -73,45 +72,52 @@
</dependency>
</dependencies>
+ <profiles>
+ <profile>
+ <id>yarn-alpha</id>
+ <modules>
+ <module>alpha</module>
+ </modules>
+ </profile>
+
+ <profile>
+ <id>yarn</id>
+ <modules>
+ <module>stable</module>
+ </modules>
+ </profile>
+ </profiles>
+
<build>
- <outputDirectory>target/scala-${scala.version}/classes</outputDirectory>
- <testOutputDirectory>target/scala-${scala.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <configuration>
- <shadedArtifactAttached>false</shadedArtifactAttached>
- <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar</outputFile>
- <artifactSet>
- <includes>
- <include>*:*</include>
- </includes>
- </artifactSet>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <excludes>
- <exclude>META-INF/*.SF</exclude>
- <exclude>META-INF/*.DSA</exclude>
- <exclude>META-INF/*.RSA</exclude>
- </excludes>
- </filter>
- </filters>
- </configuration>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
- <phase>package</phase>
+ <id>add-scala-sources</id>
+ <phase>generate-sources</phase>
+ <goals>
+ <goal>add-source</goal>
+ </goals>
+ <configuration>
+ <sources>
+ <source>src/main/scala</source>
+ <source>../common/src/main/scala</source>
+ </sources>
+ </configuration>
+ </execution>
+ <execution>
+ <id>add-scala-test-sources</id>
+ <phase>generate-test-sources</phase>
<goals>
- <goal>shade</goal>
+ <goal>add-test-source</goal>
</goals>
<configuration>
- <transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
- <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
- <resource>reference.conf</resource>
- </transformer>
- </transformers>
+ <sources>
+ <source>src/test/scala</source>
+ <source>../common/src/test/scala</source>
+ </sources>
</configuration>
</execution>
</executions>
@@ -150,12 +156,16 @@
<artifactId>scalatest-maven-plugin</artifactId>
<configuration>
<environmentVariables>
- <SPARK_HOME>${basedir}/..</SPARK_HOME>
+ <SPARK_HOME>${basedir}/../..</SPARK_HOME>
<SPARK_TESTING>1</SPARK_TESTING>
<SPARK_CLASSPATH>${spark.classpath}</SPARK_CLASSPATH>
</environmentVariables>
</configuration>
</plugin>
</plugins>
+
+ <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+ <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
</build>
+
</project>
diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml
new file mode 100644
index 0000000000..62fe3e2742
--- /dev/null
+++ b/yarn/stable/pom.xml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ ~ 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.
+ -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>yarn-parent_2.10</artifactId>
+ <version>0.9.0-incubating-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-yarn_2.10</artifactId>
+ <packaging>jar</packaging>
+ <name>Spark Project YARN Stable API</name>
+
+</project>
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
new file mode 100644
index 0000000000..69ae14ce83
--- /dev/null
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -0,0 +1,432 @@
+/*
+ * 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.yarn
+
+import java.io.IOException
+import java.net.Socket
+import java.util.concurrent.CopyOnWriteArrayList
+import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+
+import scala.collection.JavaConversions._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.util.ShutdownHookManager
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
+
+import org.apache.spark.{SparkConf, SparkContext, Logging}
+import org.apache.spark.util.Utils
+
+
+class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
+ sparkConf: SparkConf) extends Logging {
+
+ def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+ this(args, new Configuration(), sparkConf)
+
+ def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
+
+ private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+ private var appAttemptId: ApplicationAttemptId = _
+ private var userThread: Thread = _
+ private val fs = FileSystem.get(yarnConf)
+
+ private var yarnAllocator: YarnAllocationHandler = _
+ private var isFinished: Boolean = false
+ private var uiAddress: String = _
+ private val maxAppAttempts: Int = conf.getInt(
+ YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)
+ private var isLastAMRetry: Boolean = true
+ private var amClient: AMRMClient[ContainerRequest] = _
+
+ // Default to numWorkers * 2, with minimum of 3
+ private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures",
+ math.max(args.numWorkers * 2, 3))
+
+ def run() {
+ // Setup the directories so things go to YARN approved directories rather
+ // than user specified and /tmp.
+ System.setProperty("spark.local.dir", getLocalDirs())
+
+ // set the web ui port to be ephemeral for yarn so we don't conflict with
+ // other spark processes running on the same box
+ System.setProperty("spark.ui.port", "0")
+
+ // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using.
+ ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30)
+
+ appAttemptId = getApplicationAttemptId()
+ isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts
+ amClient = AMRMClient.createAMRMClient()
+ amClient.init(yarnConf)
+ amClient.start()
+
+ // 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)
+ // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf)
+
+ ApplicationMaster.register(this)
+
+ // Start the user's JAR
+ userThread = startUserClass()
+
+ // 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.
+ 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()
+
+ // Wait for the user class to Finish
+ userThread.join()
+
+ System.exit(0)
+ }
+
+ /** Get the Yarn approved local directories. */
+ private def getLocalDirs(): String = {
+ // Hadoop 0.23 and 2.x have different Environment variable names for the
+ // local dirs, so lets check both. We assume one of the 2 is set.
+ // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X
+ val localDirs = Option(System.getenv("YARN_LOCAL_DIRS"))
+ .getOrElse(Option(System.getenv("LOCAL_DIRS"))
+ .getOrElse(""))
+
+ if (localDirs.isEmpty()) {
+ throw new Exception("Yarn Local dirs can't be empty")
+ }
+ localDirs
+ }
+
+ private def getApplicationAttemptId(): ApplicationAttemptId = {
+ val envs = System.getenv()
+ val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
+ val containerId = ConverterUtils.toContainerId(containerIdString)
+ val appAttemptId = containerId.getApplicationAttemptId()
+ logInfo("ApplicationAttemptId: " + appAttemptId)
+ appAttemptId
+ }
+
+ private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+ logInfo("Registering the ApplicationMaster")
+ amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
+ }
+
+ private def startUserClass(): Thread = {
+ logInfo("Starting the user JAR in a separate Thread")
+ val mainMethod = Class.forName(
+ args.userClass,
+ false /* initialize */ ,
+ Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
+ val t = new Thread {
+ override def run() {
+ var successed = false
+ try {
+ // Copy
+ var mainArgs: Array[String] = new Array[String](args.userArgs.size)
+ args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size)
+ mainMethod.invoke(null, mainArgs)
+ // some job script has "System.exit(0)" at the end, for example SparkPi, SparkLR
+ // userThread will stop here unless it has uncaught exception thrown out
+ // It need shutdown hook to set SUCCEEDED
+ successed = true
+ } finally {
+ logDebug("finishing main")
+ isLastAMRetry = true
+ if (successed) {
+ ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+ } else {
+ ApplicationMaster.this.finishApplicationMaster(FinalApplicationStatus.FAILED)
+ }
+ }
+ }
+ }
+ t.start()
+ t
+ }
+
+ // This need to happen before allocateWorkers()
+ private def waitForSparkContextInitialized() {
+ logInfo("Waiting for Spark context initialization")
+ try {
+ var sparkContext: SparkContext = null
+ ApplicationMaster.sparkContextRef.synchronized {
+ var numTries = 0
+ val waitTime = 10000L
+ val maxNumTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10)
+ while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) {
+ logInfo("Waiting for Spark context initialization ... " + numTries)
+ numTries = numTries + 1
+ ApplicationMaster.sparkContextRef.wait(waitTime)
+ }
+ sparkContext = ApplicationMaster.sparkContextRef.get()
+ assert(sparkContext != null || numTries >= maxNumTries)
+
+ if (sparkContext != null) {
+ uiAddress = sparkContext.ui.appUIAddress
+ this.yarnAllocator = YarnAllocationHandler.newAllocator(
+ yarnConf,
+ amClient,
+ appAttemptId,
+ args,
+ sparkContext.preferredNodeLocationData,
+ sparkContext.getConf)
+ } else {
+ logWarning("Unable to retrieve SparkContext inspite of waiting for %d, maxNumTries = %d".
+ format(numTries * waitTime, maxNumTries))
+ this.yarnAllocator = YarnAllocationHandler.newAllocator(
+ yarnConf,
+ amClient,
+ appAttemptId,
+ args,
+ sparkContext.getConf)
+ }
+ }
+ } finally {
+ // In case of exceptions, etc - ensure that count is at least 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?
+ // TODO: Handle container failure
+ yarnAllocator.addResourceRequests(args.numWorkers)
+ // Exits the loop if the user thread exits.
+ while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) {
+ if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+ finishApplicationMaster(FinalApplicationStatus.FAILED,
+ "max number of worker failures reached")
+ }
+ yarnAllocator.allocateResources()
+ ApplicationMaster.incrementAllocatorLoop(1)
+ Thread.sleep(100)
+ }
+ } finally {
+ // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT,
+ // so that the loop in ApplicationMaster#sparkContextInitialized() breaks.
+ ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT)
+ }
+ logInfo("All workers have launched.")
+
+ // Launch a progress reporter thread, else the app will get killed after expiration
+ // (def: 10mins) timeout.
+ if (userThread.isAlive) {
+ // Ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapses.
+ val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+
+ // we want to be reasonably responsive without causing too many requests to RM.
+ val schedulerInterval =
+ sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000)
+
+
+ // must be <= timeoutInterval / 2.
+ val interval = math.min(timeoutInterval / 2, schedulerInterval)
+
+ launchReporterThread(interval)
+ }
+ }
+
+ private def launchReporterThread(_sleepTime: Long): Thread = {
+ val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
+
+ val t = new Thread {
+ override def run() {
+ while (userThread.isAlive) {
+ if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) {
+ finishApplicationMaster(FinalApplicationStatus.FAILED,
+ "max number of worker failures reached")
+ }
+ val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning -
+ yarnAllocator.getNumPendingAllocate
+ if (missingWorkerCount > 0) {
+ logInfo("Allocating %d containers to make up for (potentially) lost containers".
+ format(missingWorkerCount))
+ yarnAllocator.addResourceRequests(missingWorkerCount)
+ }
+ sendProgress()
+ Thread.sleep(sleepTime)
+ }
+ }
+ }
+ // Setting to daemon status, though this is usually not a good idea.
+ t.setDaemon(true)
+ t.start()
+ logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+ t
+ }
+
+ private def sendProgress() {
+ logDebug("Sending progress")
+ // Simulated with an allocate request with no nodes requested.
+ yarnAllocator.allocateResources()
+ }
+
+ /*
+ def printContainers(containers: List[Container]) = {
+ for (container <- containers) {
+ logInfo("Launching shell command on a new container."
+ + ", containerId=" + container.getId()
+ + ", containerNode=" + container.getNodeId().getHost()
+ + ":" + container.getNodeId().getPort()
+ + ", containerNodeURI=" + container.getNodeHttpAddress()
+ + ", containerState" + container.getState()
+ + ", containerResourceMemory"
+ + container.getResource().getMemory())
+ }
+ }
+ */
+
+ def finishApplicationMaster(status: FinalApplicationStatus, diagnostics: String = "") {
+ synchronized {
+ if (isFinished) {
+ return
+ }
+ isFinished = true
+ }
+
+ logInfo("finishApplicationMaster with " + status)
+ // Set tracking URL to empty since we don't have a history server.
+ amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */)
+ }
+
+ /**
+ * Clean up the staging directory.
+ */
+ private def cleanupStagingDir() {
+ var stagingDirPath: Path = null
+ try {
+ val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean
+ if (!preserveFiles) {
+ stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR"))
+ if (stagingDirPath == null) {
+ logError("Staging directory is null")
+ return
+ }
+ logInfo("Deleting staging directory " + stagingDirPath)
+ fs.delete(stagingDirPath, true)
+ }
+ } catch {
+ case ioe: IOException =>
+ logError("Failed to cleanup staging dir " + stagingDirPath, ioe)
+ }
+ }
+
+ // 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()
+ }
+ }
+
+}
+
+object ApplicationMaster {
+ // Number of times to wait for the allocator loop to complete.
+ // Each loop iteration waits for 100ms, so maximum of 3 seconds.
+ // This is to ensure that we have reasonable number of containers before we start
+ // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be
+ // optimal as more containers are available. Might need to handle this better.
+ private val ALLOCATOR_LOOP_WAIT_COUNT = 30
+
+ private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]()
+
+ val sparkContextRef: AtomicReference[SparkContext] =
+ new AtomicReference[SparkContext](null /* initialValue */)
+
+ val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0)
+
+ def incrementAllocatorLoop(by: Int) {
+ val count = yarnAllocatorLoop.getAndAdd(by)
+ if (count >= ALLOCATOR_LOOP_WAIT_COUNT) {
+ yarnAllocatorLoop.synchronized {
+ // to wake threads off wait ...
+ yarnAllocatorLoop.notifyAll()
+ }
+ }
+ }
+
+ def register(master: ApplicationMaster) {
+ applicationMasters.add(master)
+ }
+
+ // TODO(harvey): See whether this should be discarded - it isn't used anywhere atm...
+ def sparkContextInitialized(sc: SparkContext): Boolean = {
+ var modified = false
+ sparkContextRef.synchronized {
+ modified = sparkContextRef.compareAndSet(null, sc)
+ sparkContextRef.notifyAll()
+ }
+
+ // 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 the Client from declaring failure even though we exited properly.
+ // Note that this will unfortunately not properly clean up the staging files because it gets
+ // called too late, after the filesystem is already shutdown.
+ if (modified) {
+ Runtime.getRuntime().addShutdownHook(new Thread with Logging {
+ // This is not only logs, but also ensures that log system is initialized for this instance
+ // when we are actually 'run'-ing.
+ logInfo("Adding shutdown hook for context " + sc)
+
+ override def run() {
+ logInfo("Invoking sc stop from shutdown hook")
+ sc.stop()
+ // Best case ...
+ for (master <- applicationMasters) {
+ master.finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+ }
+ }
+ })
+ }
+
+ // Wait for initialization to complete and atleast 'some' nodes can get allocated.
+ yarnAllocatorLoop.synchronized {
+ while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) {
+ yarnAllocatorLoop.wait(1000L)
+ }
+ }
+ modified
+ }
+
+ def main(argStrings: Array[String]) {
+ val args = new ApplicationMasterArguments(argStrings)
+ new ApplicationMaster(args).run()
+ }
+}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
new file mode 100644
index 0000000000..be323d7783
--- /dev/null
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -0,0 +1,525 @@
+/*
+ * 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.yarn
+
+import java.net.{InetAddress, UnknownHostException, URI}
+import java.nio.ByteBuffer
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{Apps, Records}
+
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.util.Utils
+import org.apache.spark.deploy.SparkHadoopUtil
+
+
+/**
+ * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The
+ * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster,
+ * which will launch a Spark master process and negotiate resources throughout its duration.
+ */
+class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
+ extends YarnClientImpl with Logging {
+
+ def this(args: ClientArguments, sparkConf: SparkConf) =
+ this(args, new Configuration(), sparkConf)
+
+ def this(args: ClientArguments) = this(args, new SparkConf())
+
+ var rpc: YarnRPC = YarnRPC.create(conf)
+ val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+ val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+ private val SPARK_STAGING: String = ".sparkStaging"
+ private val distCacheMgr = new ClientDistributedCacheManager()
+
+ // Staging directory is private! -> rwx--------
+ val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short)
+ // App files are world-wide readable and owner writable -> rw-r--r--
+ val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short)
+
+ def runApp(): ApplicationId = {
+ validateArgs()
+ // Initialize and start the client service.
+ init(yarnConf)
+ start()
+
+ // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers).
+ logClusterResourceDetails()
+
+ // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM)
+ // interface).
+
+ // Get a new client application.
+ val newApp = super.createApplication()
+ val newAppResponse = newApp.getNewApplicationResponse()
+ val appId = newAppResponse.getApplicationId()
+
+ verifyClusterResources(newAppResponse)
+
+ // Set up resource and environment variables.
+ val appStagingDir = getAppStagingDir(appId)
+ val localResources = prepareLocalResources(appStagingDir)
+ val launchEnv = setupLaunchEnv(localResources, appStagingDir)
+ val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv)
+
+ // Set up an application submission context.
+ val appContext = newApp.getApplicationSubmissionContext()
+ appContext.setApplicationName(args.appName)
+ appContext.setQueue(args.amQueue)
+ appContext.setAMContainerSpec(amContainer)
+
+ // Memory for the ApplicationMaster.
+ val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
+ memoryResource.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+ appContext.setResource(memoryResource)
+
+ // Finally, submit and monitor the application.
+ submitApp(appContext)
+ appId
+ }
+
+ def run() {
+ val appId = runApp()
+ monitorApplication(appId)
+ System.exit(0)
+ }
+
+ // TODO(harvey): This could just go in ClientArguments.
+ def validateArgs() = {
+ Map(
+ (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
+ (args.userJar == null) -> "Error: You must specify a user jar!",
+ (args.userClass == null) -> "Error: You must specify a user class!",
+ (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!",
+ (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" +
+ "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
+ (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" +
+ "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString)
+ ).foreach { case(cond, errStr) =>
+ if (cond) {
+ logError(errStr)
+ args.printUsageAndExit(1)
+ }
+ }
+ }
+
+ def getAppStagingDir(appId: ApplicationId): String = {
+ SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
+ }
+
+ def logClusterResourceDetails() {
+ val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
+ logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " +
+ clusterMetrics.getNumNodeManagers)
+
+ val queueInfo: QueueInfo = super.getQueueInfo(args.amQueue)
+ logInfo( """Queue info ... queueName: %s, queueCurrentCapacity: %s, queueMaxCapacity: %s,
+ queueApplicationCount = %s, queueChildQueueCount = %s""".format(
+ queueInfo.getQueueName,
+ queueInfo.getCurrentCapacity,
+ queueInfo.getMaximumCapacity,
+ queueInfo.getApplications.size,
+ queueInfo.getChildQueues.size))
+ }
+
+ def verifyClusterResources(app: GetNewApplicationResponse) = {
+ val maxMem = app.getMaximumResourceCapability().getMemory()
+ logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
+
+ // If we have requested more then the clusters max for a single resource then exit.
+ if (args.workerMemory > maxMem) {
+ logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.".
+ format(args.workerMemory, maxMem))
+ System.exit(1)
+ }
+ val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
+ if (amMem > maxMem) {
+ logError("Required AM memory (%d) is above the max threshold (%d) of this cluster".
+ format(args.amMemory, maxMem))
+ System.exit(1)
+ }
+
+ // We could add checks to make sure the entire cluster has enough resources but that involves
+ // getting all the node reports and computing ourselves.
+ }
+
+ /** See if two file systems are the same or not. */
+ private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
+ val srcUri = srcFs.getUri()
+ val dstUri = destFs.getUri()
+ if (srcUri.getScheme() == null) {
+ return false
+ }
+ if (!srcUri.getScheme().equals(dstUri.getScheme())) {
+ return false
+ }
+ var srcHost = srcUri.getHost()
+ var dstHost = dstUri.getHost()
+ if ((srcHost != null) && (dstHost != null)) {
+ try {
+ srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+ dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
+ } catch {
+ case e: UnknownHostException =>
+ return false
+ }
+ if (!srcHost.equals(dstHost)) {
+ return false
+ }
+ } else if (srcHost == null && dstHost != null) {
+ return false
+ } else if (srcHost != null && dstHost == null) {
+ return false
+ }
+ //check for ports
+ if (srcUri.getPort() != dstUri.getPort()) {
+ return false
+ }
+ return true
+ }
+
+ /** Copy the file into HDFS if needed. */
+ private def copyRemoteFile(
+ dstDir: Path,
+ originalPath: Path,
+ replication: Short,
+ setPerms: Boolean = false): Path = {
+ val fs = FileSystem.get(conf)
+ val remoteFs = originalPath.getFileSystem(conf)
+ var newPath = originalPath
+ if (! compareFs(remoteFs, fs)) {
+ newPath = new Path(dstDir, originalPath.getName())
+ logInfo("Uploading " + originalPath + " to " + newPath)
+ FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
+ fs.setReplication(newPath, replication)
+ if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
+ }
+ // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+ // version shows the specific version in the distributed cache configuration
+ val qualPath = fs.makeQualified(newPath)
+ val fc = FileContext.getFileContext(qualPath.toUri(), conf)
+ val destPath = fc.resolvePath(qualPath)
+ destPath
+ }
+
+ def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
+ logInfo("Preparing Local resources")
+ // Upload Spark and the application JAR to the remote file system if necessary. Add them as
+ // local resources to the application master.
+ val fs = FileSystem.get(conf)
+
+ val delegTokenRenewer = Master.getMasterPrincipal(conf)
+ if (UserGroupInformation.isSecurityEnabled()) {
+ if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+ logError("Can't get Master Kerberos principal for use as renewer")
+ System.exit(1)
+ }
+ }
+ val dst = new Path(fs.getHomeDirectory(), appStagingDir)
+ val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort
+
+ if (UserGroupInformation.isSecurityEnabled()) {
+ val dstFs = dst.getFileSystem(conf)
+ dstFs.addDelegationTokens(delegTokenRenewer, credentials)
+ }
+
+ val localResources = HashMap[String, LocalResource]()
+ FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
+
+ val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+
+ Map(
+ Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
+ Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")
+ ).foreach { case(destName, _localPath) =>
+ val localPath: String = if (_localPath != null) _localPath.trim() else ""
+ if (! localPath.isEmpty()) {
+ var localURI = new URI(localPath)
+ // If not specified assume these are in the local filesystem to keep behavior like Hadoop
+ if (localURI.getScheme() == null) {
+ localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
+ }
+ val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
+ val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ destName, statCache)
+ }
+ }
+
+ // Handle jars local to the ApplicationMaster.
+ if ((args.addJars != null) && (!args.addJars.isEmpty())){
+ args.addJars.split(',').foreach { case file: String =>
+ val localURI = new URI(file.trim())
+ val localPath = new Path(localURI)
+ val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+ val destPath = copyRemoteFile(dst, localPath, replication)
+ // Only add the resource to the Spark ApplicationMaster.
+ val appMasterOnly = true
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ linkname, statCache, appMasterOnly)
+ }
+ }
+
+ // Handle any distributed cache files
+ if ((args.files != null) && (!args.files.isEmpty())){
+ args.files.split(',').foreach { case file: String =>
+ val localURI = new URI(file.trim())
+ val localPath = new Path(localURI)
+ val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+ val destPath = copyRemoteFile(dst, localPath, replication)
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ linkname, statCache)
+ }
+ }
+
+ // Handle any distributed cache archives
+ if ((args.archives != null) && (!args.archives.isEmpty())) {
+ args.archives.split(',').foreach { case file:String =>
+ val localURI = new URI(file.trim())
+ val localPath = new Path(localURI)
+ val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+ val destPath = copyRemoteFile(dst, localPath, replication)
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
+ linkname, statCache)
+ }
+ }
+
+ UserGroupInformation.getCurrentUser().addCredentials(credentials)
+ localResources
+ }
+
+ def setupLaunchEnv(
+ localResources: HashMap[String, LocalResource],
+ stagingDir: String): HashMap[String, String] = {
+ logInfo("Setting up the launch environment")
+ val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
+
+ val env = new HashMap[String, String]()
+
+ Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
+ env("SPARK_YARN_MODE") = "true"
+ env("SPARK_YARN_STAGING_DIR") = stagingDir
+
+ // Set the environment variables to be passed on to the Workers.
+ distCacheMgr.setDistFilesEnv(env)
+ distCacheMgr.setDistArchivesEnv(env)
+
+ // Allow users to specify some environment variables.
+ Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
+
+ // Add each SPARK_* key to the environment.
+ System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+
+ env
+ }
+
+ def userArgsToString(clientArgs: ClientArguments): String = {
+ val prefix = " --args "
+ val args = clientArgs.userArgs
+ val retval = new StringBuilder()
+ for (arg <- args) {
+ retval.append(prefix).append(" '").append(arg).append("' ")
+ }
+ retval.toString
+ }
+
+ def createContainerLaunchContext(
+ newApp: GetNewApplicationResponse,
+ localResources: HashMap[String, LocalResource],
+ env: HashMap[String, String]): ContainerLaunchContext = {
+ logInfo("Setting up container launch context")
+ val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
+ amContainer.setLocalResources(localResources)
+ amContainer.setEnvironment(env)
+
+ // TODO: Need a replacement for the following code to fix -Xmx?
+ // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
+ // var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
+ // ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
+ // YarnAllocationHandler.MEMORY_OVERHEAD)
+
+ // Extra options for the JVM
+ var JAVA_OPTS = ""
+
+ // Add Xmx for AM memory
+ JAVA_OPTS += "-Xmx" + args.amMemory + "m"
+
+ val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
+ JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir
+
+ // TODO: Remove once cpuset version is pushed out.
+ // The context is, default gc for server class machines ends up using all cores to do gc -
+ // hence if there are multiple containers in same node, Spark GC affects all other containers'
+ // performance (which can be that of other Spark containers)
+ // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
+ // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
+ // of cores on a node.
+ val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
+ java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
+ if (useConcurrentAndIncrementalGC) {
+ // In our expts, using (default) throughput collector has severe perf ramifications in
+ // multi-tenant machines
+ JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+ JAVA_OPTS += " -XX:+CMSIncrementalMode "
+ JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+ }
+
+ if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+ JAVA_OPTS += " " + env("SPARK_JAVA_OPTS")
+ }
+
+ // Command for the ApplicationMaster
+ var javaCommand = "java"
+ val javaHome = System.getenv("JAVA_HOME")
+ if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+ javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+ }
+
+ val commands = List[String](
+ javaCommand +
+ " -server " +
+ JAVA_OPTS +
+ " " + args.amClass +
+ " --class " + args.userClass +
+ " --jar " + args.userJar +
+ userArgsToString(args) +
+ " --worker-memory " + args.workerMemory +
+ " --worker-cores " + args.workerCores +
+ " --num-workers " + args.numWorkers +
+ " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+ " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+ logInfo("Command for starting the Spark ApplicationMaster: " + commands(0))
+ amContainer.setCommands(commands)
+
+ // Setup security tokens.
+ val dob = new DataOutputBuffer()
+ credentials.writeTokenStorageToStream(dob)
+ amContainer.setTokens(ByteBuffer.wrap(dob.getData()))
+
+ amContainer
+ }
+
+ def submitApp(appContext: ApplicationSubmissionContext) = {
+ // Submit the application to the applications manager.
+ logInfo("Submitting application to ASM")
+ super.submitApplication(appContext)
+ }
+
+ def monitorApplication(appId: ApplicationId): Boolean = {
+ val interval = sparkConf.getLong("spark.yarn.report.interval", 1000)
+
+ while (true) {
+ Thread.sleep(interval)
+ val report = super.getApplicationReport(appId)
+
+ logInfo("Application report from ASM: \n" +
+ "\t application identifier: " + appId.toString() + "\n" +
+ "\t appId: " + appId.getId() + "\n" +
+ "\t clientToAMToken: " + report.getClientToAMToken() + "\n" +
+ "\t appDiagnostics: " + report.getDiagnostics() + "\n" +
+ "\t appMasterHost: " + report.getHost() + "\n" +
+ "\t appQueue: " + report.getQueue() + "\n" +
+ "\t appMasterRpcPort: " + report.getRpcPort() + "\n" +
+ "\t appStartTime: " + report.getStartTime() + "\n" +
+ "\t yarnAppState: " + report.getYarnApplicationState() + "\n" +
+ "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" +
+ "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" +
+ "\t appUser: " + report.getUser()
+ )
+
+ val state = report.getYarnApplicationState()
+ val dsStatus = report.getFinalApplicationStatus()
+ if (state == YarnApplicationState.FINISHED ||
+ state == YarnApplicationState.FAILED ||
+ state == YarnApplicationState.KILLED) {
+ return true
+ }
+ }
+ true
+ }
+}
+
+object Client {
+ val SPARK_JAR: String = "spark.jar"
+ val APP_JAR: String = "app.jar"
+ val LOG4J_PROP: String = "log4j.properties"
+
+ def main(argStrings: Array[String]) {
+ // Set an env variable indicating we are running in YARN mode.
+ // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -
+ // see Client#setupLaunchEnv().
+ System.setProperty("SPARK_YARN_MODE", "true")
+ val sparkConf = new SparkConf()
+ val args = new ClientArguments(argStrings, sparkConf)
+
+ new Client(args, sparkConf).run()
+ }
+
+ // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
+ def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) {
+ for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim)
+ }
+ }
+
+ def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+ // If log4j present, ensure ours overrides all others
+ if (addLog4j) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + LOG4J_PROP)
+ }
+ // Normally the users app.jar is last in case conflicts with spark jars
+ val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false")
+ .toBoolean
+ if (userClasspathFirst) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + APP_JAR)
+ }
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + SPARK_JAR)
+ Client.populateHadoopClasspath(conf, env)
+
+ if (!userClasspathFirst) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + APP_JAR)
+ }
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + "*")
+ }
+}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala
new file mode 100644
index 0000000000..49248a8516
--- /dev/null
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.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.deploy.yarn
+
+import java.net.Socket
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.net.NetUtils
+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.util.{ConverterUtils, Records}
+import akka.actor._
+import akka.remote._
+import akka.actor.Terminated
+import org.apache.spark.{SparkConf, SparkContext, Logging}
+import org.apache.spark.util.{Utils, AkkaUtils}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.scheduler.SplitInfo
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+
+class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf)
+ extends Logging {
+
+ def this(args: ApplicationMasterArguments, sparkConf: SparkConf) =
+ this(args, new Configuration(), sparkConf)
+
+ def this(args: ApplicationMasterArguments) = this(args, new SparkConf())
+
+ private var appAttemptId: ApplicationAttemptId = _
+ private var reporterThread: Thread = _
+ private val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+ private var yarnAllocator: YarnAllocationHandler = _
+ private var driverClosed:Boolean = false
+
+ private var amClient: AMRMClient[ContainerRequest] = _
+
+ val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0,
+ conf = sparkConf)._1
+ var actor: ActorRef = _
+
+ // This actor just working as a monitor to watch on Driver Actor.
+ class MonitorActor(driverUrl: String) extends Actor {
+
+ var driver: ActorSelection = _
+
+ override def preStart() {
+ logInfo("Listen to driver: " + driverUrl)
+ driver = context.actorSelection(driverUrl)
+ // Send a hello message thus the connection is actually established, thus we can monitor Lifecycle Events.
+ driver ! "Hello"
+ context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent])
+ }
+
+ override def receive = {
+ case x: DisassociatedEvent =>
+ logInfo(s"Driver terminated or disconnected! Shutting down. $x")
+ driverClosed = true
+ }
+ }
+
+ def run() {
+
+ amClient = AMRMClient.createAMRMClient()
+ amClient.init(yarnConf)
+ amClient.start()
+
+ appAttemptId = getApplicationAttemptId()
+ registerApplicationMaster()
+
+ waitForSparkMaster()
+
+ // Allocate all containers
+ allocateWorkers()
+
+ // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout
+ // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse.
+
+ val timeoutInterval = yarnConf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 120000)
+ // must be <= timeoutInterval/ 2.
+ // On other hand, also ensure that we are reasonably responsive without causing too many requests to RM.
+ // so atleast 1 minute or timeoutInterval / 10 - whichever is higher.
+ val interval = math.min(timeoutInterval / 2, math.max(timeoutInterval / 10, 60000L))
+ reporterThread = launchReporterThread(interval)
+
+ // Wait for the reporter thread to Finish.
+ reporterThread.join()
+
+ finishApplicationMaster(FinalApplicationStatus.SUCCEEDED)
+ actorSystem.shutdown()
+
+ logInfo("Exited")
+ System.exit(0)
+ }
+
+ private def getApplicationAttemptId(): ApplicationAttemptId = {
+ val envs = System.getenv()
+ val containerIdString = envs.get(ApplicationConstants.Environment.CONTAINER_ID.name())
+ val containerId = ConverterUtils.toContainerId(containerIdString)
+ val appAttemptId = containerId.getApplicationAttemptId()
+ logInfo("ApplicationAttemptId: " + appAttemptId)
+ appAttemptId
+ }
+
+ private def registerApplicationMaster(): RegisterApplicationMasterResponse = {
+ logInfo("Registering the ApplicationMaster")
+ // TODO:(Raymond) Find out Spark UI address and fill in here?
+ amClient.registerApplicationMaster(Utils.localHostName(), 0, "")
+ }
+
+ private def waitForSparkMaster() {
+ logInfo("Waiting for Spark driver to be reachable.")
+ var driverUp = false
+ val hostport = args.userArgs(0)
+ val (driverHost, driverPort) = Utils.parseHostPort(hostport)
+ while(!driverUp) {
+ try {
+ val socket = new Socket(driverHost, driverPort)
+ socket.close()
+ logInfo("Driver now available: %s:%s".format(driverHost, driverPort))
+ driverUp = true
+ } catch {
+ case e: Exception =>
+ logError("Failed to connect to driver at %s:%s, retrying ...".
+ format(driverHost, driverPort))
+ Thread.sleep(100)
+ }
+ }
+ sparkConf.set("spark.driver.host", driverHost)
+ sparkConf.set("spark.driver.port", driverPort.toString)
+
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+ driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+ actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM")
+ }
+
+
+ private def allocateWorkers() {
+
+ // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now.
+ val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] =
+ scala.collection.immutable.Map()
+
+ yarnAllocator = YarnAllocationHandler.newAllocator(
+ yarnConf,
+ amClient,
+ appAttemptId,
+ args,
+ preferredNodeLocationData,
+ sparkConf)
+
+ logInfo("Allocating " + args.numWorkers + " workers.")
+ // Wait until all containers have finished
+ // TODO: This is a bit ugly. Can we make it nicer?
+ // TODO: Handle container failure
+
+ yarnAllocator.addResourceRequests(args.numWorkers)
+ while (yarnAllocator.getNumWorkersRunning < args.numWorkers) {
+ yarnAllocator.allocateResources()
+ Thread.sleep(100)
+ }
+
+ logInfo("All workers have launched.")
+
+ }
+
+ // TODO: We might want to extend this to allocate more containers in case they die !
+ private def launchReporterThread(_sleepTime: Long): Thread = {
+ val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime
+
+ val t = new Thread {
+ override def run() {
+ while (!driverClosed) {
+ val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning -
+ yarnAllocator.getNumPendingAllocate
+ if (missingWorkerCount > 0) {
+ logInfo("Allocating %d containers to make up for (potentially) lost containers".
+ format(missingWorkerCount))
+ yarnAllocator.addResourceRequests(missingWorkerCount)
+ }
+ sendProgress()
+ Thread.sleep(sleepTime)
+ }
+ }
+ }
+ // setting to daemon status, though this is usually not a good idea.
+ t.setDaemon(true)
+ t.start()
+ logInfo("Started progress reporter thread - sleep time : " + sleepTime)
+ t
+ }
+
+ private def sendProgress() {
+ logDebug("Sending progress")
+ // simulated with an allocate request with no nodes requested ...
+ yarnAllocator.allocateResources()
+ }
+
+ def finishApplicationMaster(status: FinalApplicationStatus) {
+ logInfo("finish ApplicationMaster with " + status)
+ amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */)
+ }
+
+}
+
+
+object WorkerLauncher {
+ def main(argStrings: Array[String]) {
+ val args = new ApplicationMasterArguments(argStrings)
+ new WorkerLauncher(args).run()
+ }
+}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
new file mode 100644
index 0000000000..b7699050bb
--- /dev/null
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -0,0 +1,210 @@
+/*
+ * 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.yarn
+
+import java.net.URI
+import java.nio.ByteBuffer
+import java.security.PrivilegedExceptionAction
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.client.api.NMClient
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
+
+import org.apache.spark.{SparkConf, Logging}
+
+
+class WorkerRunnable(
+ container: Container,
+ conf: Configuration,
+ sparkConf: SparkConf,
+ masterAddress: String,
+ slaveId: String,
+ hostname: String,
+ workerMemory: Int,
+ workerCores: Int)
+ extends Runnable with Logging {
+
+ var rpc: YarnRPC = YarnRPC.create(conf)
+ var nmClient: NMClient = _
+ val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
+
+ def run = {
+ logInfo("Starting Worker Container")
+ nmClient = NMClient.createNMClient()
+ nmClient.init(yarnConf)
+ nmClient.start()
+ startContainer
+ }
+
+ def startContainer = {
+ logInfo("Setting up ContainerLaunchContext")
+
+ val ctx = Records.newRecord(classOf[ContainerLaunchContext])
+ .asInstanceOf[ContainerLaunchContext]
+
+ val localResources = prepareLocalResources
+ ctx.setLocalResources(localResources)
+
+ val env = prepareEnvironment
+ ctx.setEnvironment(env)
+
+ // Extra options for the JVM
+ var JAVA_OPTS = ""
+ // Set the JVM memory
+ val workerMemoryString = workerMemory + "m"
+ JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " "
+ if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+ JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
+ }
+
+ 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 node, spark gc effects all other containers
+ // performance (which can also be other spark containers)
+ // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+ // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+ // of cores on a node.
+/*
+ else {
+ // If no java_opts specified, default to using -XX:+CMSIncrementalMode
+ // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
+ // want to mess with it.
+ // In our expts, using (default) throughput collector has severe perf ramnifications in
+ // multi-tennent machines
+ // The options are based on
+ // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
+ JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+ JAVA_OPTS += " -XX:+CMSIncrementalMode "
+ JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+ }
+*/
+
+ val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+ val dob = new DataOutputBuffer()
+ credentials.writeTokenStorageToStream(dob)
+ ctx.setTokens(ByteBuffer.wrap(dob.getData()))
+
+ var javaCommand = "java"
+ val javaHome = System.getenv("JAVA_HOME")
+ if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+ javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+ }
+
+ val commands = List[String](javaCommand +
+ " -server " +
+ // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
+ // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
+ // an inconsistent state.
+ // TODO: If the OOM is not recoverable by rescheduling it on different node, then do
+ // 'something' to fail job ... akin to blacklisting trackers in mapred ?
+ " -XX:OnOutOfMemoryError='kill %p' " +
+ JAVA_OPTS +
+ " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
+ masterAddress + " " +
+ slaveId + " " +
+ hostname + " " +
+ workerCores +
+ " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+ " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+ logInfo("Setting up worker with commands: " + commands)
+ ctx.setCommands(commands)
+
+ // Send the start request to the ContainerManager
+ nmClient.startContainer(container, ctx)
+ }
+
+ private def setupDistributedCache(
+ file: String,
+ rtype: LocalResourceType,
+ localResources: HashMap[String, LocalResource],
+ timestamp: String,
+ size: String,
+ vis: String) = {
+ val uri = new URI(file)
+ val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+ amJarRsrc.setType(rtype)
+ amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
+ 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 localResources = HashMap[String, LocalResource]()
+
+ 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(',')
+ val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',')
+ for( i <- 0 to distFiles.length - 1) {
+ setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
+ fileSizes(i), visibilities(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(',')
+ val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',')
+ for( i <- 0 to distArchives.length - 1) {
+ setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources,
+ timeStamps(i), fileSizes(i), visibilities(i))
+ }
+ }
+
+ logInfo("Prepared Local resources " + localResources)
+ localResources
+ }
+
+ def prepareEnvironment: HashMap[String, String] = {
+ val env = new HashMap[String, String]()
+
+ Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+
+ // Allow users to specify some environment variables
+ Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
+
+ System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+ env
+ }
+
+}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
new file mode 100644
index 0000000000..738ff986d8
--- /dev/null
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
@@ -0,0 +1,695 @@
+/*
+ * 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.yarn
+
+import java.lang.{Boolean => JBoolean}
+import java.util.{Collections, Set => JSet}
+import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection
+import scala.collection.JavaConversions._
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.Utils
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.api.ApplicationMasterProtocol
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId
+import org.apache.hadoop.yarn.api.records.{Container, ContainerId, ContainerStatus}
+import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest}
+import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse}
+import org.apache.hadoop.yarn.client.api.AMRMClient
+import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.hadoop.yarn.util.{RackResolver, Records}
+
+
+object AllocationType extends Enumeration {
+ type AllocationType = Value
+ val HOST, RACK, ANY = Value
+}
+
+// TODO:
+// Too many params.
+// Needs to be mt-safe
+// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should
+// make it more proactive and decoupled.
+
+// Note that right now, we assume all node asks as uniform in terms of capabilities and priority
+// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for
+// more info on how we are requesting for containers.
+private[yarn] class YarnAllocationHandler(
+ val conf: Configuration,
+ val amClient: AMRMClient[ContainerRequest],
+ val appAttemptId: ApplicationAttemptId,
+ val maxWorkers: Int,
+ val workerMemory: Int,
+ val workerCores: Int,
+ val preferredHostToCount: Map[String, Int],
+ val preferredRackToCount: Map[String, Int],
+ val sparkConf: SparkConf)
+ extends Logging {
+ // These three are locked on allocatedHostToContainersMap. Complementary data structures
+ // allocatedHostToContainersMap : containers which are running : host, Set<containerid>
+ // allocatedContainerToHostMap: container to host mapping.
+ private val allocatedHostToContainersMap =
+ new HashMap[String, collection.mutable.Set[ContainerId]]()
+
+ private val allocatedContainerToHostMap = new HashMap[ContainerId, String]()
+
+ // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an
+ // allocated node)
+ // As with the two data structures above, tightly coupled with them, and to be locked on
+ // allocatedHostToContainersMap
+ private val allocatedRackCount = new HashMap[String, Int]()
+
+ // Containers which have been released.
+ private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]()
+ // Containers to be released in next request to RM
+ private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean]
+
+ // Number of container requests that have been sent to, but not yet allocated by the
+ // ApplicationMaster.
+ private val numPendingAllocate = new AtomicInteger()
+ private val numWorkersRunning = new AtomicInteger()
+ // Used to generate a unique id per worker
+ private val workerIdCounter = new AtomicInteger()
+ private val lastResponseId = new AtomicInteger()
+ private val numWorkersFailed = new AtomicInteger()
+
+ def getNumPendingAllocate: Int = numPendingAllocate.intValue
+
+ def getNumWorkersRunning: Int = numWorkersRunning.intValue
+
+ def getNumWorkersFailed: Int = numWorkersFailed.intValue
+
+ def isResourceConstraintSatisfied(container: Container): Boolean = {
+ container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+ }
+
+ def releaseContainer(container: Container) {
+ val containerId = container.getId
+ pendingReleaseContainers.put(containerId, true)
+ amClient.releaseAssignedContainer(containerId)
+ }
+
+ def allocateResources() {
+ // We have already set the container request. Poll the ResourceManager for a response.
+ // This doubles as a heartbeat if there are no pending container requests.
+ val progressIndicator = 0.1f
+ val allocateResponse = amClient.allocate(progressIndicator)
+
+ val allocatedContainers = allocateResponse.getAllocatedContainers()
+ if (allocatedContainers.size > 0) {
+ var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size)
+
+ if (numPendingAllocateNow < 0) {
+ numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow)
+ }
+
+ logDebug("""
+ Allocated containers: %d
+ Current worker count: %d
+ Containers released: %s
+ Containers to-be-released: %s
+ Cluster resources: %s
+ """.format(
+ allocatedContainers.size,
+ numWorkersRunning.get(),
+ releasedContainerList,
+ pendingReleaseContainers,
+ allocateResponse.getAvailableResources))
+
+ val hostToContainers = new HashMap[String, ArrayBuffer[Container]]()
+
+ for (container <- allocatedContainers) {
+ if (isResourceConstraintSatisfied(container)) {
+ // Add the accepted `container` to the host's list of already accepted,
+ // allocated containers
+ val host = container.getNodeId.getHost
+ val containersForHost = hostToContainers.getOrElseUpdate(host,
+ new ArrayBuffer[Container]())
+ containersForHost += container
+ } else {
+ // Release container, since it doesn't satisfy resource constraints.
+ releaseContainer(container)
+ }
+ }
+
+ // Find the appropriate containers to use.
+ // TODO: Cleanup this group-by...
+ val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
+ val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]()
+ val offRackContainers = new HashMap[String, ArrayBuffer[Container]]()
+
+ for (candidateHost <- hostToContainers.keySet) {
+ val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0)
+ val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost)
+
+ val remainingContainersOpt = hostToContainers.get(candidateHost)
+ assert(remainingContainersOpt.isDefined)
+ var remainingContainers = remainingContainersOpt.get
+
+ if (requiredHostCount >= remainingContainers.size) {
+ // Since we have <= required containers, add all remaining containers to
+ // `dataLocalContainers`.
+ dataLocalContainers.put(candidateHost, remainingContainers)
+ // There are no more free containers remaining.
+ remainingContainers = null
+ } else if (requiredHostCount > 0) {
+ // Container list has more containers than we need for data locality.
+ // Split the list into two: one based on the data local container count,
+ // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining
+ // containers.
+ val (dataLocal, remaining) = remainingContainers.splitAt(
+ remainingContainers.size - requiredHostCount)
+ dataLocalContainers.put(candidateHost, dataLocal)
+
+ // Invariant: remainingContainers == remaining
+
+ // YARN has a nasty habit of allocating a ton of containers on a host - discourage this.
+ // Add each container in `remaining` to list of containers to release. If we have an
+ // insufficient number of containers, then the next allocation cycle will reallocate
+ // (but won't treat it as data local).
+ // TODO(harvey): Rephrase this comment some more.
+ for (container <- remaining) releaseContainer(container)
+ remainingContainers = null
+ }
+
+ // For rack local containers
+ if (remainingContainers != null) {
+ val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+ if (rack != null) {
+ val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0)
+ val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) -
+ rackLocalContainers.getOrElse(rack, List()).size
+
+ if (requiredRackCount >= remainingContainers.size) {
+ // Add all remaining containers to to `dataLocalContainers`.
+ dataLocalContainers.put(rack, remainingContainers)
+ remainingContainers = null
+ } else if (requiredRackCount > 0) {
+ // Container list has more containers that we need for data locality.
+ // Split the list into two: one based on the data local container count,
+ // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining
+ // containers.
+ val (rackLocal, remaining) = remainingContainers.splitAt(
+ remainingContainers.size - requiredRackCount)
+ val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack,
+ new ArrayBuffer[Container]())
+
+ existingRackLocal ++= rackLocal
+
+ remainingContainers = remaining
+ }
+ }
+ }
+
+ if (remainingContainers != null) {
+ // Not all containers have been consumed - add them to the list of off-rack containers.
+ offRackContainers.put(candidateHost, remainingContainers)
+ }
+ }
+
+ // Now that we have split the containers into various groups, go through them in order:
+ // first host-local, then rack-local, and finally off-rack.
+ // Note that the list we create below tries to ensure that not all containers end up within
+ // a host if there is a sufficiently large number of hosts/containers.
+ val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size)
+ allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers)
+ allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers)
+ allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers)
+
+ // Run each of the allocated containers.
+ for (container <- allocatedContainersToProcess) {
+ val numWorkersRunningNow = numWorkersRunning.incrementAndGet()
+ val workerHostname = container.getNodeId.getHost
+ val containerId = container.getId
+
+ val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+ assert(container.getResource.getMemory >= workerMemoryOverhead)
+
+ if (numWorkersRunningNow > maxWorkers) {
+ logInfo("""Ignoring container %s at host %s, since we already have the required number of
+ containers for it.""".format(containerId, workerHostname))
+ releaseContainer(container)
+ numWorkersRunning.decrementAndGet()
+ } else {
+ val workerId = workerIdCounter.incrementAndGet().toString
+ val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format(
+ sparkConf.get("spark.driver.host"),
+ sparkConf.get("spark.driver.port"),
+ CoarseGrainedSchedulerBackend.ACTOR_NAME)
+
+ logInfo("Launching container %s for on host %s".format(containerId, workerHostname))
+
+ // To be safe, remove the container from `pendingReleaseContainers`.
+ pendingReleaseContainers.remove(containerId)
+
+ val rack = YarnAllocationHandler.lookupRack(conf, workerHostname)
+ allocatedHostToContainersMap.synchronized {
+ val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname,
+ new HashSet[ContainerId]())
+
+ containerSet += containerId
+ allocatedContainerToHostMap.put(containerId, workerHostname)
+
+ if (rack != null) {
+ allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1)
+ }
+ }
+ logInfo("Launching WorkerRunnable. driverUrl: %s, workerHostname: %s".format(driverUrl, workerHostname))
+ val workerRunnable = new WorkerRunnable(
+ container,
+ conf,
+ sparkConf,
+ driverUrl,
+ workerId,
+ workerHostname,
+ workerMemory,
+ workerCores)
+ new Thread(workerRunnable).start()
+ }
+ }
+ logDebug("""
+ Finished allocating %s containers (from %s originally).
+ Current number of workers running: %d,
+ releasedContainerList: %s,
+ pendingReleaseContainers: %s
+ """.format(
+ allocatedContainersToProcess,
+ allocatedContainers,
+ numWorkersRunning.get(),
+ releasedContainerList,
+ pendingReleaseContainers))
+ }
+
+ val completedContainers = allocateResponse.getCompletedContainersStatuses()
+ if (completedContainers.size > 0) {
+ logDebug("Completed %d containers".format(completedContainers.size))
+
+ for (completedContainer <- completedContainers) {
+ val containerId = completedContainer.getContainerId
+
+ if (pendingReleaseContainers.containsKey(containerId)) {
+ // YarnAllocationHandler already marked the container for release, so remove it from
+ // `pendingReleaseContainers`.
+ pendingReleaseContainers.remove(containerId)
+ } else {
+ // Decrement the number of workers running. The next iteration of the ApplicationMaster's
+ // reporting thread will take care of allocating.
+ numWorkersRunning.decrementAndGet()
+ logInfo("Completed container %s (state: %s, exit status: %s)".format(
+ containerId,
+ completedContainer.getState,
+ completedContainer.getExitStatus()))
+ // Hadoop 2.2.X added a ContainerExitStatus we should switch to use
+ // there are some exit status' we shouldn't necessarily count against us, but for
+ // now I think its ok as none of the containers are expected to exit
+ if (completedContainer.getExitStatus() != 0) {
+ logInfo("Container marked as failed: " + containerId)
+ numWorkersFailed.incrementAndGet()
+ }
+ }
+
+ allocatedHostToContainersMap.synchronized {
+ if (allocatedContainerToHostMap.containsKey(containerId)) {
+ val hostOpt = allocatedContainerToHostMap.get(containerId)
+ assert(hostOpt.isDefined)
+ val host = hostOpt.get
+
+ val containerSetOpt = allocatedHostToContainersMap.get(host)
+ assert(containerSetOpt.isDefined)
+ val containerSet = containerSetOpt.get
+
+ containerSet.remove(containerId)
+ if (containerSet.isEmpty) {
+ allocatedHostToContainersMap.remove(host)
+ } else {
+ allocatedHostToContainersMap.update(host, containerSet)
+ }
+
+ allocatedContainerToHostMap.remove(containerId)
+
+ // TODO: Move this part outside the synchronized block?
+ val rack = YarnAllocationHandler.lookupRack(conf, host)
+ if (rack != null) {
+ val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1
+ if (rackCount > 0) {
+ allocatedRackCount.put(rack, rackCount)
+ } else {
+ allocatedRackCount.remove(rack)
+ }
+ }
+ }
+ }
+ }
+ logDebug("""
+ Finished processing %d completed containers.
+ Current number of workers running: %d,
+ releasedContainerList: %s,
+ pendingReleaseContainers: %s
+ """.format(
+ completedContainers.size,
+ numWorkersRunning.get(),
+ releasedContainerList,
+ pendingReleaseContainers))
+ }
+ }
+
+ def createRackResourceRequests(
+ hostContainers: ArrayBuffer[ContainerRequest]
+ ): ArrayBuffer[ContainerRequest] = {
+ // Generate modified racks and new set of hosts under it before issuing requests.
+ val rackToCounts = new HashMap[String, Int]()
+
+ for (container <- hostContainers) {
+ val candidateHost = container.getNodes.last
+ assert(YarnAllocationHandler.ANY_HOST != candidateHost)
+
+ val rack = YarnAllocationHandler.lookupRack(conf, candidateHost)
+ if (rack != null) {
+ var count = rackToCounts.getOrElse(rack, 0)
+ count += 1
+ rackToCounts.put(rack, count)
+ }
+ }
+
+ val requestedContainers = new ArrayBuffer[ContainerRequest](rackToCounts.size)
+ for ((rack, count) <- rackToCounts) {
+ requestedContainers ++= createResourceRequests(
+ AllocationType.RACK,
+ rack,
+ count,
+ YarnAllocationHandler.PRIORITY)
+ }
+
+ requestedContainers
+ }
+
+ def allocatedContainersOnHost(host: String): Int = {
+ var retval = 0
+ allocatedHostToContainersMap.synchronized {
+ retval = allocatedHostToContainersMap.getOrElse(host, Set()).size
+ }
+ retval
+ }
+
+ def allocatedContainersOnRack(rack: String): Int = {
+ var retval = 0
+ allocatedHostToContainersMap.synchronized {
+ retval = allocatedRackCount.getOrElse(rack, 0)
+ }
+ retval
+ }
+
+ def addResourceRequests(numWorkers: Int) {
+ val containerRequests: List[ContainerRequest] =
+ if (numWorkers <= 0 || preferredHostToCount.isEmpty) {
+ logDebug("numWorkers: " + numWorkers + ", host preferences: " +
+ preferredHostToCount.isEmpty)
+ createResourceRequests(
+ AllocationType.ANY,
+ resource = null,
+ numWorkers,
+ YarnAllocationHandler.PRIORITY).toList
+ } else {
+ // Request for all hosts in preferred nodes and for numWorkers -
+ // candidates.size, request by default allocation policy.
+ val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size)
+ for ((candidateHost, candidateCount) <- preferredHostToCount) {
+ val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost)
+
+ if (requiredCount > 0) {
+ hostContainerRequests ++= createResourceRequests(
+ AllocationType.HOST,
+ candidateHost,
+ requiredCount,
+ YarnAllocationHandler.PRIORITY)
+ }
+ }
+ val rackContainerRequests: List[ContainerRequest] = createRackResourceRequests(
+ hostContainerRequests).toList
+
+ val anyContainerRequests = createResourceRequests(
+ AllocationType.ANY,
+ resource = null,
+ numWorkers,
+ YarnAllocationHandler.PRIORITY)
+
+ val containerRequestBuffer = new ArrayBuffer[ContainerRequest](
+ hostContainerRequests.size + rackContainerRequests.size() + anyContainerRequests.size)
+
+ containerRequestBuffer ++= hostContainerRequests
+ containerRequestBuffer ++= rackContainerRequests
+ containerRequestBuffer ++= anyContainerRequests
+ containerRequestBuffer.toList
+ }
+
+ for (request <- containerRequests) {
+ amClient.addContainerRequest(request)
+ }
+
+ if (numWorkers > 0) {
+ numPendingAllocate.addAndGet(numWorkers)
+ logInfo("Will Allocate %d worker containers, each with %d memory".format(
+ numWorkers,
+ (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)))
+ } else {
+ logDebug("Empty allocation request ...")
+ }
+
+ for (request <- containerRequests) {
+ val nodes = request.getNodes
+ var hostStr = if (nodes == null || nodes.isEmpty) {
+ "Any"
+ } else {
+ nodes.last
+ }
+ logInfo("Container request (host: %s, priority: %s, capability: %s".format(
+ hostStr,
+ request.getPriority().getPriority,
+ request.getCapability))
+ }
+ }
+
+ private def createResourceRequests(
+ requestType: AllocationType.AllocationType,
+ resource: String,
+ numWorkers: Int,
+ priority: Int
+ ): ArrayBuffer[ContainerRequest] = {
+
+ // If hostname is specified, then we need at least two requests - node local and rack local.
+ // There must be a third request, which is ANY. That will be specially handled.
+ requestType match {
+ case AllocationType.HOST => {
+ assert(YarnAllocationHandler.ANY_HOST != resource)
+ val hostname = resource
+ val nodeLocal = constructContainerRequests(
+ Array(hostname),
+ racks = null,
+ numWorkers,
+ priority)
+
+ // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler.
+ YarnAllocationHandler.populateRackInfo(conf, hostname)
+ nodeLocal
+ }
+ case AllocationType.RACK => {
+ val rack = resource
+ constructContainerRequests(hosts = null, Array(rack), numWorkers, priority)
+ }
+ case AllocationType.ANY => constructContainerRequests(
+ hosts = null, racks = null, numWorkers, priority)
+ case _ => throw new IllegalArgumentException(
+ "Unexpected/unsupported request type: " + requestType)
+ }
+ }
+
+ private def constructContainerRequests(
+ hosts: Array[String],
+ racks: Array[String],
+ numWorkers: Int,
+ priority: Int
+ ): ArrayBuffer[ContainerRequest] = {
+
+ val memoryResource = Records.newRecord(classOf[Resource])
+ memoryResource.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+
+ val prioritySetting = Records.newRecord(classOf[Priority])
+ prioritySetting.setPriority(priority)
+
+ val requests = new ArrayBuffer[ContainerRequest]()
+ for (i <- 0 until numWorkers) {
+ requests += new ContainerRequest(memoryResource, hosts, racks, prioritySetting)
+ }
+ requests
+ }
+}
+
+object YarnAllocationHandler {
+
+ val ANY_HOST = "*"
+ // All requests are issued with same priority : we do not (yet) have any distinction between
+ // request types (like map/reduce in hadoop for example)
+ val PRIORITY = 1
+
+ // Additional memory overhead - in mb.
+ val MEMORY_OVERHEAD = 384
+
+ // Host to rack map - saved from allocation requests. We are expecting this not to change.
+ // Note that it is possible for this to change : and ResurceManager will indicate that to us via
+ // update response to allocate. But we are punting on handling that for now.
+ private val hostToRack = new ConcurrentHashMap[String, String]()
+ private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]()
+
+
+ def newAllocator(
+ conf: Configuration,
+ amClient: AMRMClient[ContainerRequest],
+ appAttemptId: ApplicationAttemptId,
+ args: ApplicationMasterArguments,
+ sparkConf: SparkConf
+ ): YarnAllocationHandler = {
+ new YarnAllocationHandler(
+ conf,
+ amClient,
+ appAttemptId,
+ args.numWorkers,
+ args.workerMemory,
+ args.workerCores,
+ Map[String, Int](),
+ Map[String, Int](),
+ sparkConf)
+ }
+
+ def newAllocator(
+ conf: Configuration,
+ amClient: AMRMClient[ContainerRequest],
+ appAttemptId: ApplicationAttemptId,
+ args: ApplicationMasterArguments,
+ map: collection.Map[String,
+ collection.Set[SplitInfo]],
+ sparkConf: SparkConf
+ ): YarnAllocationHandler = {
+ val (hostToSplitCount, rackToSplitCount) = generateNodeToWeight(conf, map)
+ new YarnAllocationHandler(
+ conf,
+ amClient,
+ appAttemptId,
+ args.numWorkers,
+ args.workerMemory,
+ args.workerCores,
+ hostToSplitCount,
+ rackToSplitCount,
+ sparkConf)
+ }
+
+ def newAllocator(
+ conf: Configuration,
+ amClient: AMRMClient[ContainerRequest],
+ appAttemptId: ApplicationAttemptId,
+ maxWorkers: Int,
+ workerMemory: Int,
+ workerCores: Int,
+ map: collection.Map[String, collection.Set[SplitInfo]],
+ sparkConf: SparkConf
+ ): YarnAllocationHandler = {
+ val (hostToCount, rackToCount) = generateNodeToWeight(conf, map)
+ new YarnAllocationHandler(
+ conf,
+ amClient,
+ appAttemptId,
+ maxWorkers,
+ workerMemory,
+ workerCores,
+ hostToCount,
+ rackToCount,
+ sparkConf)
+ }
+
+ // A simple method to copy the split info map.
+ private def generateNodeToWeight(
+ conf: Configuration,
+ input: collection.Map[String, collection.Set[SplitInfo]]
+ ): (Map[String, Int], Map[String, Int]) = {
+
+ if (input == null) {
+ return (Map[String, Int](), Map[String, Int]())
+ }
+
+ val hostToCount = new HashMap[String, Int]
+ val rackToCount = new HashMap[String, Int]
+
+ for ((host, splits) <- input) {
+ val hostCount = hostToCount.getOrElse(host, 0)
+ hostToCount.put(host, hostCount + splits.size)
+
+ val rack = lookupRack(conf, host)
+ if (rack != null){
+ val rackCount = rackToCount.getOrElse(host, 0)
+ rackToCount.put(host, rackCount + splits.size)
+ }
+ }
+
+ (hostToCount.toMap, rackToCount.toMap)
+ }
+
+ def lookupRack(conf: Configuration, host: String): String = {
+ if (!hostToRack.contains(host)) {
+ populateRackInfo(conf, host)
+ }
+ hostToRack.get(host)
+ }
+
+ def fetchCachedHostsForRack(rack: String): Option[Set[String]] = {
+ Option(rackToHostSet.get(rack)).map { set =>
+ val convertedSet: collection.mutable.Set[String] = set
+ // TODO: Better way to get a Set[String] from JSet.
+ convertedSet.toSet
+ }
+ }
+
+ def populateRackInfo(conf: Configuration, hostname: String) {
+ Utils.checkHost(hostname)
+
+ if (!hostToRack.containsKey(hostname)) {
+ // If there are repeated failures to resolve, all to an ignore list.
+ val rackInfo = RackResolver.resolve(conf, hostname)
+ if (rackInfo != null && rackInfo.getNetworkLocation != null) {
+ val rack = rackInfo.getNetworkLocation
+ hostToRack.put(hostname, rack)
+ if (! rackToHostSet.containsKey(rack)) {
+ rackToHostSet.putIfAbsent(rack,
+ Collections.newSetFromMap(new ConcurrentHashMap[String, JBoolean]()))
+ }
+ rackToHostSet.get(rack).add(hostname)
+
+ // TODO(harvey): Figure out what this comment means...
+ // Since RackResolver caches, we are disabling this for now ...
+ } /* else {
+ // right ? Else we will keep calling rack resolver in case we cant resolve rack info ...
+ hostToRack.put(hostname, null)
+ } */
+ }
+ }
+}