aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--.gitignore3
-rw-r--r--README.md60
-rw-r--r--bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala5
-rw-r--r--bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala16
-rw-r--r--bagel/src/test/resources/log4j.properties10
-rw-r--r--bagel/src/test/scala/bagel/BagelSuite.scala2
-rwxr-xr-xbin/slaves.sh3
-rwxr-xr-xbin/spark-daemon.sh3
-rwxr-xr-xbin/start-slaves.sh18
-rwxr-xr-xconf/spark-env.sh.template29
-rw-r--r--core/lib/mesos-0.9.0.jarbin264708 -> 0 bytes
-rw-r--r--core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala7
-rw-r--r--core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala9
-rw-r--r--core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala7
-rw-r--r--core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala10
-rw-r--r--core/src/main/scala/spark/Accumulators.scala88
-rw-r--r--core/src/main/scala/spark/Aggregator.scala43
-rw-r--r--core/src/main/scala/spark/BlockStoreShuffleFetcher.scala45
-rw-r--r--core/src/main/scala/spark/BoundedMemoryCache.scala6
-rw-r--r--core/src/main/scala/spark/Cache.scala10
-rw-r--r--core/src/main/scala/spark/CacheTracker.scala57
-rw-r--r--core/src/main/scala/spark/ClosureCleaner.scala6
-rw-r--r--core/src/main/scala/spark/Dependency.scala41
-rw-r--r--core/src/main/scala/spark/DoubleRDDFunctions.scala18
-rw-r--r--core/src/main/scala/spark/FetchFailedException.scala2
-rw-r--r--core/src/main/scala/spark/HadoopWriter.scala17
-rw-r--r--core/src/main/scala/spark/HttpFileServer.scala47
-rw-r--r--core/src/main/scala/spark/HttpServer.scala4
-rw-r--r--core/src/main/scala/spark/JavaSerializer.scala12
-rw-r--r--core/src/main/scala/spark/KryoSerializer.scala31
-rw-r--r--core/src/main/scala/spark/Logging.scala24
-rw-r--r--core/src/main/scala/spark/MapOutputTracker.scala158
-rw-r--r--core/src/main/scala/spark/PairRDDFunctions.scala328
-rw-r--r--core/src/main/scala/spark/ParallelCollection.scala4
-rw-r--r--core/src/main/scala/spark/Partitioner.scala15
-rw-r--r--core/src/main/scala/spark/RDD.scala254
-rw-r--r--core/src/main/scala/spark/SequenceFileRDDFunctions.scala13
-rw-r--r--core/src/main/scala/spark/ShuffleFetcher.scala12
-rw-r--r--core/src/main/scala/spark/ShuffleManager.scala98
-rw-r--r--core/src/main/scala/spark/ShuffledRDD.scala51
-rw-r--r--core/src/main/scala/spark/SizeEstimator.scala28
-rw-r--r--core/src/main/scala/spark/SoftReferenceCache.scala2
-rw-r--r--core/src/main/scala/spark/SparkContext.scala337
-rw-r--r--core/src/main/scala/spark/SparkEnv.scala86
-rw-r--r--core/src/main/scala/spark/TaskEndReason.scala14
-rw-r--r--core/src/main/scala/spark/TaskState.scala2
-rw-r--r--core/src/main/scala/spark/Utils.scala139
-rw-r--r--core/src/main/scala/spark/api/java/JavaDoubleRDD.scala34
-rw-r--r--core/src/main/scala/spark/api/java/JavaPairRDD.scala243
-rw-r--r--core/src/main/scala/spark/api/java/JavaRDD.scala25
-rw-r--r--core/src/main/scala/spark/api/java/JavaRDDLike.scala128
-rw-r--r--core/src/main/scala/spark/api/java/JavaSparkContext.scala117
-rw-r--r--core/src/main/scala/spark/api/java/StorageLevels.java20
-rw-r--r--core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java3
-rw-r--r--core/src/main/scala/spark/api/java/function/DoubleFunction.java3
-rw-r--r--core/src/main/scala/spark/api/java/function/FlatMapFunction.scala3
-rw-r--r--core/src/main/scala/spark/api/java/function/Function.java5
-rw-r--r--core/src/main/scala/spark/api/java/function/Function2.java3
-rw-r--r--core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java4
-rw-r--r--core/src/main/scala/spark/api/java/function/PairFunction.java3
-rw-r--r--core/src/main/scala/spark/api/java/function/VoidFunction.scala3
-rw-r--r--core/src/main/scala/spark/api/java/function/WrappedFunction1.scala2
-rw-r--r--core/src/main/scala/spark/api/java/function/WrappedFunction2.scala2
-rw-r--r--core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala79
-rw-r--r--core/src/main/scala/spark/broadcast/Broadcast.scala23
-rw-r--r--core/src/main/scala/spark/broadcast/BroadcastFactory.scala4
-rw-r--r--core/src/main/scala/spark/broadcast/HttpBroadcast.scala52
-rw-r--r--core/src/main/scala/spark/broadcast/MultiTracker.scala53
-rw-r--r--core/src/main/scala/spark/broadcast/SourceInfo.scala9
-rw-r--r--core/src/main/scala/spark/broadcast/TreeBroadcast.scala75
-rw-r--r--core/src/main/scala/spark/deploy/Command.scala2
-rw-r--r--core/src/main/scala/spark/deploy/DeployMessage.scala29
-rw-r--r--core/src/main/scala/spark/deploy/ExecutorState.scala2
-rw-r--r--core/src/main/scala/spark/deploy/JobDescription.scala2
-rw-r--r--core/src/main/scala/spark/deploy/LocalSparkCluster.scala58
-rw-r--r--core/src/main/scala/spark/deploy/client/Client.scala14
-rw-r--r--core/src/main/scala/spark/deploy/client/ClientListener.scala2
-rw-r--r--core/src/main/scala/spark/deploy/client/TestClient.scala2
-rw-r--r--core/src/main/scala/spark/deploy/client/TestExecutor.scala2
-rw-r--r--core/src/main/scala/spark/deploy/master/ExecutorInfo.scala2
-rw-r--r--core/src/main/scala/spark/deploy/master/JobInfo.scala10
-rw-r--r--core/src/main/scala/spark/deploy/master/JobState.scala4
-rw-r--r--core/src/main/scala/spark/deploy/master/Master.scala41
-rw-r--r--core/src/main/scala/spark/deploy/master/MasterArguments.scala4
-rw-r--r--core/src/main/scala/spark/deploy/master/MasterWebUI.scala5
-rw-r--r--core/src/main/scala/spark/deploy/master/WorkerInfo.scala2
-rw-r--r--core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala62
-rw-r--r--core/src/main/scala/spark/deploy/worker/Worker.scala24
-rw-r--r--core/src/main/scala/spark/deploy/worker/WorkerArguments.scala13
-rw-r--r--core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala3
-rw-r--r--core/src/main/scala/spark/executor/Executor.scala84
-rw-r--r--core/src/main/scala/spark/executor/ExecutorBackend.scala2
-rw-r--r--core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala14
-rw-r--r--core/src/main/scala/spark/executor/MesosExecutorBackend.scala4
-rw-r--r--core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala4
-rw-r--r--core/src/main/scala/spark/network/Connection.scala32
-rw-r--r--core/src/main/scala/spark/network/ConnectionManager.scala19
-rw-r--r--core/src/main/scala/spark/network/ConnectionManagerTest.scala2
-rw-r--r--core/src/main/scala/spark/network/Message.scala19
-rw-r--r--core/src/main/scala/spark/network/ReceiverTest.scala2
-rw-r--r--core/src/main/scala/spark/network/SenderTest.scala2
-rw-r--r--core/src/main/scala/spark/package.scala15
-rw-r--r--core/src/main/scala/spark/partial/ApproximateActionListener.scala2
-rw-r--r--core/src/main/scala/spark/partial/ApproximateEvaluator.scala2
-rw-r--r--core/src/main/scala/spark/partial/CountEvaluator.scala2
-rw-r--r--core/src/main/scala/spark/partial/GroupedCountEvaluator.scala2
-rw-r--r--core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala2
-rw-r--r--core/src/main/scala/spark/partial/GroupedSumEvaluator.scala2
-rw-r--r--core/src/main/scala/spark/partial/MeanEvaluator.scala2
-rw-r--r--core/src/main/scala/spark/partial/StudentTCacher.scala2
-rw-r--r--core/src/main/scala/spark/partial/SumEvaluator.scala2
-rw-r--r--core/src/main/scala/spark/rdd/BlockRDD.scala (renamed from core/src/main/scala/spark/BlockRDD.scala)12
-rw-r--r--core/src/main/scala/spark/rdd/CartesianRDD.scala (renamed from core/src/main/scala/spark/CartesianRDD.scala)11
-rw-r--r--core/src/main/scala/spark/rdd/CoGroupedRDD.scala (renamed from core/src/main/scala/spark/CoGroupedRDD.scala)44
-rw-r--r--core/src/main/scala/spark/rdd/CoalescedRDD.scala47
-rw-r--r--core/src/main/scala/spark/rdd/FilteredRDD.scala12
-rw-r--r--core/src/main/scala/spark/rdd/FlatMappedRDD.scala16
-rw-r--r--core/src/main/scala/spark/rdd/GlommedRDD.scala12
-rw-r--r--core/src/main/scala/spark/rdd/HadoopRDD.scala (renamed from core/src/main/scala/spark/HadoopRDD.scala)10
-rw-r--r--core/src/main/scala/spark/rdd/MapPartitionsRDD.scala19
-rw-r--r--core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala21
-rw-r--r--core/src/main/scala/spark/rdd/MappedRDD.scala16
-rw-r--r--core/src/main/scala/spark/rdd/NewHadoopRDD.scala (renamed from core/src/main/scala/spark/NewHadoopRDD.scala)24
-rw-r--r--core/src/main/scala/spark/rdd/PipedRDD.scala (renamed from core/src/main/scala/spark/PipedRDD.scala)8
-rw-r--r--core/src/main/scala/spark/rdd/SampledRDD.scala (renamed from core/src/main/scala/spark/SampledRDD.scala)31
-rw-r--r--core/src/main/scala/spark/rdd/ShuffledRDD.scala40
-rw-r--r--core/src/main/scala/spark/rdd/UnionRDD.scala (renamed from core/src/main/scala/spark/UnionRDD.scala)12
-rw-r--r--core/src/main/scala/spark/scheduler/ActiveJob.scala3
-rw-r--r--core/src/main/scala/spark/scheduler/DAGScheduler.scala127
-rw-r--r--core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala13
-rw-r--r--core/src/main/scala/spark/scheduler/JobListener.scala2
-rw-r--r--core/src/main/scala/spark/scheduler/JobResult.scala6
-rw-r--r--core/src/main/scala/spark/scheduler/JobWaiter.scala2
-rw-r--r--core/src/main/scala/spark/scheduler/MapStatus.scala27
-rw-r--r--core/src/main/scala/spark/scheduler/ResultTask.scala2
-rw-r--r--core/src/main/scala/spark/scheduler/ShuffleMapTask.scala90
-rw-r--r--core/src/main/scala/spark/scheduler/Stage.scala18
-rw-r--r--core/src/main/scala/spark/scheduler/Task.scala86
-rw-r--r--core/src/main/scala/spark/scheduler/TaskResult.scala1
-rw-r--r--core/src/main/scala/spark/scheduler/TaskScheduler.scala2
-rw-r--r--core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala5
-rw-r--r--core/src/main/scala/spark/scheduler/TaskSet.scala4
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala30
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala2
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala1
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala22
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala14
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala5
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala2
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala1
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala13
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala1
-rw-r--r--core/src/main/scala/spark/scheduler/local/LocalScheduler.scala63
-rw-r--r--core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala22
-rw-r--r--core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala22
-rw-r--r--core/src/main/scala/spark/serializer/Serializer.scala (renamed from core/src/main/scala/spark/Serializer.scala)22
-rw-r--r--core/src/main/scala/spark/storage/BlockManager.scala729
-rw-r--r--core/src/main/scala/spark/storage/BlockManagerMaster.scala250
-rw-r--r--core/src/main/scala/spark/storage/BlockManagerWorker.scala44
-rw-r--r--core/src/main/scala/spark/storage/BlockMessage.scala14
-rw-r--r--core/src/main/scala/spark/storage/BlockMessageArray.scala8
-rw-r--r--core/src/main/scala/spark/storage/BlockStore.scala313
-rw-r--r--core/src/main/scala/spark/storage/DiskStore.scala180
-rw-r--r--core/src/main/scala/spark/storage/MemoryStore.scala218
-rw-r--r--core/src/main/scala/spark/storage/PutResult.scala9
-rw-r--r--core/src/main/scala/spark/storage/StorageLevel.scala25
-rw-r--r--core/src/main/scala/spark/util/AkkaUtils.scala10
-rw-r--r--core/src/main/scala/spark/util/ByteBufferInputStream.scala41
-rw-r--r--core/src/main/scala/spark/util/IntParam.scala2
-rw-r--r--core/src/main/scala/spark/util/MemoryParam.scala2
-rw-r--r--core/src/main/scala/spark/util/SerializableBuffer.scala1
-rw-r--r--core/src/main/scala/spark/util/StatCounter.scala23
-rw-r--r--core/src/main/twirl/spark/deploy/common/layout.scala.html (renamed from core/src/main/twirl/common/layout.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/master/executor_row.scala.html (renamed from core/src/main/twirl/masterui/executor_row.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/master/executors_table.scala.html (renamed from core/src/main/twirl/masterui/executors_table.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/master/index.scala.html (renamed from core/src/main/twirl/masterui/index.scala.html)6
-rw-r--r--core/src/main/twirl/spark/deploy/master/job_details.scala.html (renamed from core/src/main/twirl/masterui/job_details.scala.html)4
-rw-r--r--core/src/main/twirl/spark/deploy/master/job_row.scala.html (renamed from core/src/main/twirl/masterui/job_row.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/master/job_table.scala.html (renamed from core/src/main/twirl/masterui/job_table.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/master/worker_row.scala.html (renamed from core/src/main/twirl/masterui/worker_row.scala.html)7
-rw-r--r--core/src/main/twirl/spark/deploy/master/worker_table.scala.html (renamed from core/src/main/twirl/masterui/worker_table.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/worker/executor_row.scala.html (renamed from core/src/main/twirl/workerui/executor_row.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/worker/executors_table.scala.html (renamed from core/src/main/twirl/workerui/executors_table.scala.html)0
-rw-r--r--core/src/main/twirl/spark/deploy/worker/index.scala.html (renamed from core/src/main/twirl/workerui/index.scala.html)7
-rw-r--r--core/src/test/resources/log4j.properties10
-rw-r--r--core/src/test/resources/uncommons-maths-1.2.2.jarbin0 -> 49019 bytes
-rw-r--r--core/src/test/scala/spark/AccumulatorSuite.scala54
-rw-r--r--core/src/test/scala/spark/BoundedMemoryCacheSuite.scala14
-rw-r--r--core/src/test/scala/spark/BroadcastSuite.scala2
-rw-r--r--core/src/test/scala/spark/CacheTrackerSuite.scala2
-rw-r--r--core/src/test/scala/spark/DistributedSuite.scala191
-rw-r--r--core/src/test/scala/spark/FailureSuite.scala2
-rw-r--r--core/src/test/scala/spark/FileServerSuite.scala110
-rw-r--r--core/src/test/scala/spark/FileSuite.scala18
-rw-r--r--core/src/test/scala/spark/JavaAPISuite.java6
-rw-r--r--core/src/test/scala/spark/MapOutputTrackerSuite.scala25
-rw-r--r--core/src/test/scala/spark/PartitioningSuite.scala2
-rw-r--r--core/src/test/scala/spark/PipedRDDSuite.scala3
-rw-r--r--core/src/test/scala/spark/RDDSuite.scala48
-rw-r--r--core/src/test/scala/spark/ShuffleSuite.scala45
-rw-r--r--core/src/test/scala/spark/SizeEstimatorSuite.scala29
-rw-r--r--core/src/test/scala/spark/SortingSuite.scala69
-rw-r--r--core/src/test/scala/spark/ThreadingSuite.scala2
-rw-r--r--core/src/test/scala/spark/storage/BlockManagerSuite.scala254
-rw-r--r--docs/README.md34
-rw-r--r--docs/_config.yml9
-rwxr-xr-xdocs/_layouts/404.html157
-rwxr-xr-xdocs/_layouts/global.html142
-rw-r--r--docs/_plugins/copy_api_dirs.rb30
-rw-r--r--docs/api.md10
-rw-r--r--docs/bagel-programming-guide.md161
-rw-r--r--docs/configuration.md227
-rw-r--r--docs/contributing-to-spark.md26
-rw-r--r--docs/css/bootstrap-responsive.css1040
-rw-r--r--docs/css/bootstrap-responsive.min.css9
-rw-r--r--docs/css/bootstrap.css5624
-rw-r--r--docs/css/bootstrap.min.css9
-rwxr-xr-xdocs/css/main.css106
-rw-r--r--docs/css/pygments-default.css76
-rw-r--r--docs/ec2-scripts.md159
-rwxr-xr-xdocs/img/glyphicons-halflings-white.pngbin0 -> 8777 bytes
-rwxr-xr-xdocs/img/glyphicons-halflings.pngbin0 -> 12799 bytes
-rw-r--r--docs/img/spark-logo-100x40px.pngbin0 -> 3635 bytes
-rw-r--r--docs/img/spark-logo-77x40px-hd.pngbin0 -> 1904 bytes
-rw-r--r--docs/img/spark-logo-77x50px-hd.pngbin0 -> 3536 bytes
-rw-r--r--docs/index.md96
-rw-r--r--docs/java-programming-guide.md194
-rwxr-xr-xdocs/js/main.js1
-rwxr-xr-xdocs/js/vendor/bootstrap.js2027
-rwxr-xr-xdocs/js/vendor/bootstrap.min.js6
-rwxr-xr-xdocs/js/vendor/jquery-1.8.0.min.js2
-rwxr-xr-xdocs/js/vendor/modernizr-2.6.1-respond-1.1.0.min.js11
-rw-r--r--docs/quick-start.md232
-rw-r--r--docs/running-on-mesos.md59
-rw-r--r--docs/running-on-yarn.md51
-rw-r--r--docs/scala-programming-guide.md354
-rw-r--r--docs/spark-debugger.md121
-rw-r--r--docs/spark-simple-tutorial.md41
-rw-r--r--docs/spark-standalone.md152
-rw-r--r--docs/tuning.md248
-rw-r--r--ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh1
-rwxr-xr-xec2/spark_ec2.py42
-rw-r--r--examples/src/main/java/spark/examples/JavaHdfsLR.java2
-rw-r--r--examples/src/main/java/spark/examples/JavaTC.java2
-rw-r--r--examples/src/main/scala/spark/examples/BroadcastTest.scala12
-rw-r--r--examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala2
-rw-r--r--examples/src/main/scala/spark/examples/GroupByTest.scala2
-rw-r--r--examples/src/main/scala/spark/examples/MultiBroadcastTest.scala2
-rw-r--r--examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala2
-rw-r--r--examples/src/main/scala/spark/examples/SkewedGroupByTest.scala2
-rw-r--r--examples/src/main/scala/spark/examples/SparkALS.scala2
-rw-r--r--examples/src/main/scala/spark/examples/SparkLR.scala6
-rw-r--r--examples/src/main/scala/spark/examples/SparkPi.scala2
-rw-r--r--examples/src/main/scala/spark/examples/SparkTC.scala2
-rw-r--r--project/SparkBuild.scala106
-rw-r--r--project/plugins.sbt5
-rw-r--r--repl/src/main/scala/spark/repl/SparkILoop.scala2
-rw-r--r--repl/src/test/resources/log4j.properties10
-rw-r--r--repl/src/test/scala/spark/repl/ReplSuite.scala25
-rwxr-xr-xrun58
-rw-r--r--run.cmd2
-rw-r--r--run2.cmd67
-rwxr-xr-xsbt/sbt1
-rw-r--r--sbt/sbt.cmd5
-rw-r--r--spark-shell.cmd4
-rw-r--r--streaming/src/main/scala/spark/streaming/Checkpoint.scala2
-rw-r--r--streaming/src/main/scala/spark/streaming/CoGroupedDStream.scala3
-rw-r--r--streaming/src/main/scala/spark/streaming/DStream.scala3
-rw-r--r--streaming/src/main/scala/spark/streaming/FileInputDStream.scala2
-rw-r--r--streaming/src/main/scala/spark/streaming/NetworkInputDStream.scala8
-rw-r--r--streaming/src/main/scala/spark/streaming/QueueInputDStream.scala2
-rw-r--r--streaming/src/main/scala/spark/streaming/ReducedWindowedDStream.scala4
-rw-r--r--streaming/src/main/scala/spark/streaming/StateDStream.scala4
-rw-r--r--streaming/src/main/scala/spark/streaming/StreamingContext.scala4
-rw-r--r--streaming/src/main/scala/spark/streaming/WindowedDStream.scala2
-rw-r--r--streaming/src/main/scala/spark/streaming/examples/TopKWordCountRaw.scala2
-rw-r--r--streaming/src/main/scala/spark/streaming/examples/WordCount2.scala4
-rw-r--r--streaming/src/main/scala/spark/streaming/examples/WordCountRaw.scala2
-rw-r--r--streaming/src/main/scala/spark/streaming/examples/WordMax2.scala4
-rw-r--r--streaming/src/main/scala/spark/streaming/util/TestStreamReceiver3.scala4
-rw-r--r--streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala2
281 files changed, 16857 insertions, 2264 deletions
diff --git a/.gitignore b/.gitignore
index d486f58db5..f22248f40d 100644
--- a/.gitignore
+++ b/.gitignore
@@ -13,6 +13,8 @@ third_party/libmesos.dylib
conf/java-opts
conf/spark-env.sh
conf/log4j.properties
+docs/_site
+docs/api
target/
reports/
.project
@@ -28,3 +30,4 @@ project/plugins/lib_managed/
project/plugins/src_managed/
logs/
log/
+spark-tests.log
diff --git a/README.md b/README.md
index df9e73e4bd..b0fc3524fa 100644
--- a/README.md
+++ b/README.md
@@ -6,23 +6,21 @@ Lightning-Fast Cluster Computing - <http://www.spark-project.org/>
## Online Documentation
You can find the latest Spark documentation, including a programming
-guide, on the project wiki at <http://github.com/mesos/spark/wiki>. This
-file only contains basic setup instructions.
+guide, on the project webpage at <http://spark-project.org/documentation.html>.
+This README file only contains basic setup instructions.
## Building
-Spark requires Scala 2.9.1. This version has been tested with 2.9.1.final.
+Spark requires Scala 2.9.2. The project is built using Simple Build Tool (SBT),
+which is packaged with it. To build Spark and its example programs, run:
-The project is built using Simple Build Tool (SBT), which is packaged with it.
-To build Spark and its example programs, run:
+ sbt/sbt package
- sbt/sbt compile
-
-To run Spark, you will need to have Scala's bin in your `PATH`, or you
-will need to set the `SCALA_HOME` environment variable to point to where
+To run Spark, you will need to have Scala's bin directory in your `PATH`, or
+you will need to set the `SCALA_HOME` environment variable to point to where
you've installed Scala. Scala must be accessible through one of these
-methods on Mesos slave nodes as well as on the master.
+methods on your cluster's worker nodes as well as its master.
To run one of the examples, use `./run <class> <params>`. For example:
@@ -32,12 +30,12 @@ will run the Logistic Regression example locally on 2 CPUs.
Each of the example programs prints usage help if no params are given.
-All of the Spark samples take a `<host>` parameter that is the Mesos master
-to connect to. This can be a Mesos URL, or "local" to run locally with one
-thread, or "local[N]" to run locally with N threads.
+All of the Spark samples take a `<host>` parameter that is the cluster URL
+to connect to. This can be a mesos:// or spark:// URL, or "local" to run
+locally with one thread, or "local[N]" to run locally with N threads.
-## A Note About Hadoop
+## A Note About Hadoop Versions
Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported
storage systems. Because the HDFS API has changed in different versions of
@@ -48,27 +46,23 @@ of `project/SparkBuild.scala`, then rebuilding Spark.
## Configuration
-Spark can be configured through two files: `conf/java-opts` and
-`conf/spark-env.sh`.
-
-In `java-opts`, you can add flags to be passed to the JVM when running Spark.
-
-In `spark-env.sh`, you can set any environment variables you wish to be available
-when running Spark programs, such as `PATH`, `SCALA_HOME`, etc. There are also
-several Spark-specific variables you can set:
-
-- `SPARK_CLASSPATH`: Extra entries to be added to the classpath, separated by ":".
+Please refer to the "Configuration" guide in the online documentation for a
+full overview on how to configure Spark. At the minimum, you will need to
+create a `conf/spark-env.sh` script (copy `conf/spark-env.sh.template`) and
+set the following two variables:
-- `SPARK_MEM`: Memory for Spark to use, in the format used by java's `-Xmx`
- option (for example, `-Xmx200m` means 200 MB, `-Xmx1g` means 1 GB, etc).
+- `SCALA_HOME`: Location where Scala is installed.
-- `SPARK_LIBRARY_PATH`: Extra entries to add to `java.library.path` for locating
- shared libraries.
+- `MESOS_NATIVE_LIBRARY`: Your Mesos library (only needed if you want to run
+ on Mesos). For example, this might be `/usr/local/lib/libmesos.so` on Linux.
-- `SPARK_JAVA_OPTS`: Extra options to pass to JVM.
-- `MESOS_NATIVE_LIBRARY`: Your Mesos library, if you want to run on a Mesos
- cluster. For example, this might be `/usr/local/lib/libmesos.so` on Linux.
+## Contributing to Spark
-Note that `spark-env.sh` must be a shell script (it must be executable and start
-with a `#!` header to specify the shell to use).
+Contributions via GitHub pull requests are gladly accepted from their original
+author. Along with any pull requests, please state that the contribution is
+your original work and that you license the work to the project under the
+project's open source license. Whether or not you state this explicitly, by
+submitting any copyrighted material via pull request, email, or other means
+you agree to license the material under the project's open source license and
+warrant that you have the legal authority to do so.
diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
index f37ee01fd2..03843019c0 100644
--- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
+++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRank.scala
@@ -8,6 +8,11 @@ import spark.bagel.Bagel._
import scala.xml.{XML,NodeSeq}
+/**
+ * Run PageRank on XML Wikipedia dumps from http://wiki.freebase.com/wiki/WEX. Uses the "articles"
+ * files from there, which contains one line per wiki article in a tab-separated format
+ * (http://wiki.freebase.com/wiki/WEX/Documentation#articles).
+ */
object WikipediaPageRank {
def main(args: Array[String]) {
if (args.length < 5) {
diff --git a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
index ed8ace3a57..06cc8c748b 100644
--- a/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
+++ b/bagel/src/main/scala/spark/bagel/examples/WikipediaPageRankStandalone.scala
@@ -1,6 +1,7 @@
package spark.bagel.examples
import spark._
+import serializer.{DeserializationStream, SerializationStream, SerializerInstance}
import spark.SparkContext._
import spark.bagel._
@@ -33,10 +34,10 @@ object WikipediaPageRankStandalone {
val partitioner = new HashPartitioner(sc.defaultParallelism)
val links =
if (usePartitioner)
- input.map(parseArticle _).partitionBy(partitioner).cache
+ input.map(parseArticle _).partitionBy(partitioner).cache()
else
- input.map(parseArticle _).cache
- val n = links.count
+ input.map(parseArticle _).cache()
+ val n = links.count()
val defaultRank = 1.0 / n
val a = 0.15
@@ -51,7 +52,7 @@ object WikipediaPageRankStandalone {
(ranks
.filter { case (id, rank) => rank >= threshold }
.map { case (id, rank) => "%s\t%s\n".format(id, rank) }
- .collect.mkString)
+ .collect().mkString)
println(top)
val time = (System.currentTimeMillis - startTime) / 1000.0
@@ -113,7 +114,7 @@ object WikipediaPageRankStandalone {
}
}
-class WPRSerializer extends spark.Serializer {
+class WPRSerializer extends spark.serializer.Serializer {
def newInstance(): SerializerInstance = new WPRSerializerInstance()
}
@@ -142,7 +143,7 @@ class WPRSerializerInstance extends SerializerInstance {
class WPRSerializationStream(os: OutputStream) extends SerializationStream {
val dos = new DataOutputStream(os)
- def writeObject[T](t: T): Unit = t match {
+ def writeObject[T](t: T): SerializationStream = t match {
case (id: String, wrapper: ArrayBuffer[_]) => wrapper(0) match {
case links: Array[String] => {
dos.writeInt(0) // links
@@ -151,17 +152,20 @@ class WPRSerializationStream(os: OutputStream) extends SerializationStream {
for (link <- links) {
dos.writeUTF(link)
}
+ this
}
case rank: Double => {
dos.writeInt(1) // rank
dos.writeUTF(id)
dos.writeDouble(rank)
+ this
}
}
case (id: String, rank: Double) => {
dos.writeInt(2) // rank without wrapper
dos.writeUTF(id)
dos.writeDouble(rank)
+ this
}
}
diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties
index 02fe16866e..4c99e450bc 100644
--- a/bagel/src/test/resources/log4j.properties
+++ b/bagel/src/test/resources/log4j.properties
@@ -1,8 +1,10 @@
# Set everything to be logged to the console
-log4j.rootCategory=WARN, 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.SSS} %p %c{1}: %m%n
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=spark-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/bagel/src/test/scala/bagel/BagelSuite.scala b/bagel/src/test/scala/bagel/BagelSuite.scala
index 3da7152a09..ca59f46843 100644
--- a/bagel/src/test/scala/bagel/BagelSuite.scala
+++ b/bagel/src/test/scala/bagel/BagelSuite.scala
@@ -22,6 +22,8 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
test("halting by voting") {
diff --git a/bin/slaves.sh b/bin/slaves.sh
index d383e02545..030581e058 100755
--- a/bin/slaves.sh
+++ b/bin/slaves.sh
@@ -1,5 +1,8 @@
#!/usr/bin/env bash
+# This Spark deploy script is a modified version of the Apache Hadoop deploy
+# script, available under the Apache 2 license:
+#
# 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.
diff --git a/bin/spark-daemon.sh b/bin/spark-daemon.sh
index ae0b781291..0c584055c7 100755
--- a/bin/spark-daemon.sh
+++ b/bin/spark-daemon.sh
@@ -1,5 +1,8 @@
#!/usr/bin/env bash
+# This Spark deploy script is a modified version of the Apache Hadoop deploy
+# script, available under the Apache 2 license:
+#
# 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.
diff --git a/bin/start-slaves.sh b/bin/start-slaves.sh
index f348ea063f..74b70a24be 100755
--- a/bin/start-slaves.sh
+++ b/bin/start-slaves.sh
@@ -14,7 +14,21 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then
SPARK_MASTER_PORT=7077
fi
-hostname=`hostname`
-ip=`host "$hostname" | cut -d " " -f 4`
+if [ "$SPARK_MASTER_IP" = "" ]; then
+ hostname=`hostname`
+ hostouput=`host "$hostname"`
+
+ if [[ "$hostouput" == *"not found"* ]]; then
+ echo $hostouput
+ echo "Fail to identiy the IP for the master."
+ echo "Set SPARK_MASTER_IP explicitly in configuration instead."
+ exit 1
+ fi
+ ip=`host "$hostname" | cut -d " " -f 4`
+else
+ ip=$SPARK_MASTER_IP
+fi
+
+echo "Master IP: $ip"
"$bin"/spark-daemons.sh start spark.deploy.worker.Worker spark://$ip:$SPARK_MASTER_PORT \ No newline at end of file
diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template
index 64eacce8a2..6d71ec5691 100755
--- a/conf/spark-env.sh.template
+++ b/conf/spark-env.sh.template
@@ -1,21 +1,24 @@
#!/usr/bin/env bash
-# Set Spark environment variables for your site in this file. Some useful
-# variables to set are:
+# This file contains environment variables required to run Spark. Copy it as
+# spark-env.sh and edit that to configure Spark for your site. At a minimum,
+# the following two variables should be set:
# - MESOS_NATIVE_LIBRARY, to point to your Mesos native library (libmesos.so)
# - SCALA_HOME, to point to your Scala installation
+#
+# If using the standalone deploy mode, you can also set variables for it:
+# - SPARK_MASTER_IP, to bind the master to a different IP address
+# - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports
+# - SPARK_WORKER_CORES, to set the number of cores to use on this machine
+# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g)
+# - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT
+#
+# Finally, Spark also relies on the following variables, but these can be set
+# on just the *master* (i.e. in your driver program), and will automatically
+# be propagated to workers:
+# - SPARK_MEM, to change the amount of memory used per node (this should
+# be in the same format as the JVM's -Xmx option, e.g. 300m or 1g)
# - SPARK_CLASSPATH, to add elements to Spark's classpath
# - SPARK_JAVA_OPTS, to add JVM options
-# - SPARK_MEM, to change the amount of memory used per node (this should
-# be in the same format as the JVM's -Xmx option, e.g. 300m or 1g).
# - SPARK_LIBRARY_PATH, to add extra search paths for native libraries.
-# Settings used by the scripts in the bin/ directory, apply to standalone mode only.
-# Note that the same worker settings apply to all of the workers.
-# - SPARK_MASTER_IP, to bind the master to a different ip address, for example a public one (Default: local ip address)
-# - SPARK_MASTER_PORT, to start the spark master on a different port (Default: 7077)
-# - SPARK_MASTER_WEBUI_PORT, to specify a different port for the Master WebUI (Default: 8080)
-# - SPARK_WORKER_PORT, to start the spark worker on a specific port (Default: random)
-# - SPARK_WORKER_CORES, to specify the number of cores to use (Default: all available cores)
-# - SPARK_WORKER_MEMORY, to specify how much memory to use, e.g. 1000M, 2G (Default: MAX(Available - 1024MB, 512MB))
-# - SPARK_WORKER_WEBUI_PORT, to specify a different port for the Worker WebUI (Default: 8081) \ No newline at end of file
diff --git a/core/lib/mesos-0.9.0.jar b/core/lib/mesos-0.9.0.jar
deleted file mode 100644
index b7ad79bf2a..0000000000
--- a/core/lib/mesos-0.9.0.jar
+++ /dev/null
Binary files differ
diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
new file mode 100644
index 0000000000..ca9f7219de
--- /dev/null
+++ b/core/src/hadoop1/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -0,0 +1,7 @@
+package org.apache.hadoop.mapred
+
+trait HadoopMapRedUtil {
+ def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContext(conf, jobId)
+
+ def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId)
+}
diff --git a/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
new file mode 100644
index 0000000000..de7b0f81e3
--- /dev/null
+++ b/core/src/hadoop1/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -0,0 +1,9 @@
+package org.apache.hadoop.mapreduce
+
+import org.apache.hadoop.conf.Configuration
+
+trait HadoopMapReduceUtil {
+ def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContext(conf, jobId)
+
+ def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContext(conf, attemptId)
+}
diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
new file mode 100644
index 0000000000..35300cea58
--- /dev/null
+++ b/core/src/hadoop2/scala/org/apache/hadoop/mapred/HadoopMapRedUtil.scala
@@ -0,0 +1,7 @@
+package org.apache.hadoop.mapred
+
+trait HadoopMapRedUtil {
+ def newJobContext(conf: JobConf, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
+
+ def newTaskAttemptContext(conf: JobConf, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
+}
diff --git a/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
new file mode 100644
index 0000000000..7afdbff320
--- /dev/null
+++ b/core/src/hadoop2/scala/org/apache/hadoop/mapreduce/HadoopMapReduceUtil.scala
@@ -0,0 +1,10 @@
+package org.apache.hadoop.mapreduce
+
+import org.apache.hadoop.conf.Configuration
+import task.{TaskAttemptContextImpl, JobContextImpl}
+
+trait HadoopMapReduceUtil {
+ def newJobContext(conf: Configuration, jobId: JobID): JobContext = new JobContextImpl(conf, jobId)
+
+ def newTaskAttemptContext(conf: Configuration, attemptId: TaskAttemptID): TaskAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
+}
diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala
index d764ffc29d..bacd0ace37 100644
--- a/core/src/main/scala/spark/Accumulators.scala
+++ b/core/src/main/scala/spark/Accumulators.scala
@@ -3,18 +3,20 @@ package spark
import java.io._
import scala.collection.mutable.Map
+import scala.collection.generic.Growable
/**
- * A datatype that can be accumulated, i.e. has an commutative and associative +.
+ * A datatype that can be accumulated, i.e. has an commutative and associative "add" operation,
+ * but where the result type, `R`, may be different from the element type being added, `T`.
*
- * You must define how to add data, and how to merge two of these together. For some datatypes, these might be
- * the same operation (eg., a counter). In that case, you might want to use [[spark.AccumulatorParam]]. They won't
- * always be the same, though -- eg., imagine you are accumulating a set. You will add items to the set, and you
- * will union two sets together.
+ * You must define how to add data, and how to merge two of these together. For some datatypes,
+ * such as a counter, these might be the same operation. In that case, you can use the simpler
+ * [[spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are
+ * accumulating a set. You will add items to the set, and you will union two sets together.
*
* @param initialValue initial value of accumulator
- * @param param helper object defining how to add elements of type `T`
- * @tparam R the full accumulated data
+ * @param param helper object defining how to add elements of type `R` and `T`
+ * @tparam R the full accumulated data (result type)
* @tparam T partial data that can be added in
*/
class Accumulable[R, T] (
@@ -43,13 +45,29 @@ class Accumulable[R, T] (
* @param term the other Accumulable that will get merged with this
*/
def ++= (term: R) { value_ = param.addInPlace(value_, term)}
+
+ /**
+ * Access the accumulator's current value; only allowed on master.
+ */
def value = {
if (!deserialized) value_
else throw new UnsupportedOperationException("Can't read accumulator value in task")
}
- private[spark] def localValue = value_
+ /**
+ * Get the current value of this accumulator from within a task.
+ *
+ * This is NOT the global value of the accumulator. To get the global value after a
+ * completed operation on the dataset, call `value`.
+ *
+ * The typical use of this method is to directly mutate the local value, eg., to add
+ * an element to a Set.
+ */
+ def localValue = value_
+ /**
+ * Set the accumulator's value; only allowed on master.
+ */
def value_= (r: R) {
if (!deserialized) value_ = r
else throw new UnsupportedOperationException("Can't assign accumulator value in task")
@@ -67,31 +85,64 @@ class Accumulable[R, T] (
}
/**
- * Helper object defining how to accumulate values of a particular type.
+ * Helper object defining how to accumulate values of a particular type. An implicit
+ * AccumulableParam needs to be available when you create Accumulables of a specific type.
*
- * @tparam R the full accumulated data
+ * @tparam R the full accumulated data (result type)
* @tparam T partial data that can be added in
*/
trait AccumulableParam[R, T] extends Serializable {
/**
- * Add additional data to the accumulator value.
+ * Add additional data to the accumulator value. Is allowed to modify and return `r`
+ * for efficiency (to avoid allocating objects).
+ *
* @param r the current value of the accumulator
* @param t the data to be added to the accumulator
* @return the new value of the accumulator
*/
- def addAccumulator(r: R, t: T) : R
+ def addAccumulator(r: R, t: T): R
/**
- * Merge two accumulated values together
+ * Merge two accumulated values together. Is allowed to modify and return the first value
+ * for efficiency (to avoid allocating objects).
+ *
* @param r1 one set of accumulated data
* @param r2 another set of accumulated data
* @return both data sets merged together
*/
def addInPlace(r1: R, r2: R): R
+ /**
+ * Return the "zero" (identity) value for an accumulator type, given its initial value. For
+ * example, if R was a vector of N dimensions, this would return a vector of N zeroes.
+ */
def zero(initialValue: R): R
}
+private[spark]
+class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializable, T]
+ extends AccumulableParam[R,T] {
+
+ def addAccumulator(growable: R, elem: T): R = {
+ growable += elem
+ growable
+ }
+
+ def addInPlace(t1: R, t2: R): R = {
+ t1 ++= t2
+ t1
+ }
+
+ 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 spark.JavaSerializer).newInstance()
+ val copy = ser.deserialize[R](ser.serialize(initialValue))
+ copy.clear() // In case it contained stuff
+ copy
+ }
+}
+
/**
* A simpler value of [[spark.Accumulable]] where the result type being accumulated is the same
* as the types of elements being merged.
@@ -100,17 +151,18 @@ trait AccumulableParam[R, T] extends Serializable {
* @param param helper object defining how to add elements of type `T`
* @tparam T result type
*/
-class Accumulator[T](
- @transient initialValue: T,
- param: AccumulatorParam[T]) extends Accumulable[T,T](initialValue, param)
+class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T])
+ extends Accumulable[T,T](initialValue, param)
/**
* A simpler version of [[spark.AccumulableParam]] where the only datatype you can add in is the same type
- * as the accumulated value
+ * as the accumulated value. An implicit AccumulatorParam object needs to be available when you create
+ * Accumulators of a specific type.
+ *
* @tparam T type of value to accumulate
*/
trait AccumulatorParam[T] extends AccumulableParam[T, T] {
- def addAccumulator(t1: T, t2: T) : T = {
+ def addAccumulator(t1: T, t2: T): T = {
addInPlace(t1, t2)
}
}
diff --git a/core/src/main/scala/spark/Aggregator.scala b/core/src/main/scala/spark/Aggregator.scala
index 6f99270b1e..df8ce9c054 100644
--- a/core/src/main/scala/spark/Aggregator.scala
+++ b/core/src/main/scala/spark/Aggregator.scala
@@ -1,7 +1,44 @@
package spark
-class Aggregator[K, V, C] (
+import java.util.{HashMap => JHashMap}
+
+import scala.collection.JavaConversions._
+
+/** A set of functions used to aggregate data.
+ *
+ * @param createCombiner function to create the initial value of the aggregation.
+ * @param mergeValue function to merge a new value into the aggregation result.
+ * @param mergeCombiners function to merge outputs from multiple mergeValue function.
+ */
+case class Aggregator[K, V, C] (
val createCombiner: V => C,
val mergeValue: (C, V) => C,
- val mergeCombiners: (C, C) => C)
- extends Serializable
+ val mergeCombiners: (C, C) => C) {
+
+ def combineValuesByKey(iter: Iterator[(K, V)]) : Iterator[(K, C)] = {
+ val combiners = new JHashMap[K, C]
+ for ((k, v) <- iter) {
+ val oldC = combiners.get(k)
+ if (oldC == null) {
+ combiners.put(k, createCombiner(v))
+ } else {
+ combiners.put(k, mergeValue(oldC, v))
+ }
+ }
+ combiners.iterator
+ }
+
+ def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = {
+ val combiners = new JHashMap[K, C]
+ for ((k, c) <- iter) {
+ val oldC = combiners.get(k)
+ if (oldC == null) {
+ combiners.put(k, c)
+ } else {
+ combiners.put(k, mergeCombiners(oldC, c))
+ }
+ }
+ combiners.iterator
+ }
+}
+
diff --git a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
index 0bbdb4e432..86432d0127 100644
--- a/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
+++ b/core/src/main/scala/spark/BlockStoreShuffleFetcher.scala
@@ -1,52 +1,43 @@
package spark
-import java.io.EOFException
-import java.net.URL
-
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
-import spark.storage.BlockException
import spark.storage.BlockManagerId
-import it.unimi.dsi.fastutil.io.FastBufferedInputStream
-
-
-class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
- def fetch[K, V](shuffleId: Int, reduceId: Int, func: (K, V) => Unit) {
+private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
+ override def fetch[K, V](shuffleId: Int, reduceId: Int) = {
logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId))
val blockManager = SparkEnv.get.blockManager
val startTime = System.currentTimeMillis
- val addresses = SparkEnv.get.mapOutputTracker.getServerAddresses(shuffleId)
+ val statuses = SparkEnv.get.mapOutputTracker.getServerStatuses(shuffleId, reduceId)
logDebug("Fetching map output location for shuffle %d, reduce %d took %d ms".format(
shuffleId, reduceId, System.currentTimeMillis - startTime))
- val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[Int]]
- for ((address, index) <- addresses.zipWithIndex) {
- splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += index
+ val splitsByAddress = new HashMap[BlockManagerId, ArrayBuffer[(Int, Long)]]
+ for (((address, size), index) <- statuses.zipWithIndex) {
+ splitsByAddress.getOrElseUpdate(address, ArrayBuffer()) += ((index, size))
}
- val blocksByAddress: Seq[(BlockManagerId, Seq[String])] = splitsByAddress.toSeq.map {
+ val blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])] = splitsByAddress.toSeq.map {
case (address, splits) =>
- (address, splits.map(i => "shuffleid_%d_%d_%d".format(shuffleId, i, reduceId)))
+ (address, splits.map(s => ("shuffle_%d_%d_%d".format(shuffleId, s._1, reduceId), s._2)))
}
- for ((blockId, blockOption) <- blockManager.getMultiple(blocksByAddress)) {
+ def unpackBlock(blockPair: (String, Option[Iterator[Any]])) : Iterator[(K, V)] = {
+ val blockId = blockPair._1
+ val blockOption = blockPair._2
blockOption match {
case Some(block) => {
- val values = block
- for(value <- values) {
- val v = value.asInstanceOf[(K, V)]
- func(v._1, v._2)
- }
+ block.asInstanceOf[Iterator[(K, V)]]
}
case None => {
- val regex = "shuffleid_([0-9]*)_([0-9]*)_([0-9]*)".r
+ val regex = "shuffle_([0-9]*)_([0-9]*)_([0-9]*)".r
blockId match {
- case regex(shufId, mapId, reduceId) =>
- val addr = addresses(mapId.toInt)
- throw new FetchFailedException(addr, shufId.toInt, mapId.toInt, reduceId.toInt, null)
+ case regex(shufId, mapId, _) =>
+ val address = statuses(mapId.toInt)._1
+ throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, null)
case _ =>
throw new SparkException(
"Failed to get block " + blockId + ", which is not a shuffle block")
@@ -54,8 +45,6 @@ class BlockStoreShuffleFetcher extends ShuffleFetcher with Logging {
}
}
}
-
- logDebug("Fetching and merging outputs of shuffle %d, reduce %d took %d ms".format(
- shuffleId, reduceId, System.currentTimeMillis - startTime))
+ blockManager.getMultiple(blocksByAddress).flatMap(unpackBlock)
}
}
diff --git a/core/src/main/scala/spark/BoundedMemoryCache.scala b/core/src/main/scala/spark/BoundedMemoryCache.scala
index 6fe0b94297..e8392a194f 100644
--- a/core/src/main/scala/spark/BoundedMemoryCache.scala
+++ b/core/src/main/scala/spark/BoundedMemoryCache.scala
@@ -9,7 +9,7 @@ import java.util.LinkedHashMap
* some cache entries have pointers to a shared object. Nonetheless, this Cache should work well
* when most of the space is used by arrays of primitives or of simple classes.
*/
-class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging {
+private[spark] class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging {
logInfo("BoundedMemoryCache.maxBytes = " + maxBytes)
def this() {
@@ -104,9 +104,9 @@ class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging {
}
// An entry in our map; stores a cached object and its size in bytes
-case class Entry(value: Any, size: Long)
+private[spark] case class Entry(value: Any, size: Long)
-object BoundedMemoryCache {
+private[spark] object BoundedMemoryCache {
/**
* Get maximum cache capacity from system configuration
*/
diff --git a/core/src/main/scala/spark/Cache.scala b/core/src/main/scala/spark/Cache.scala
index 150fe14e2c..20d677a854 100644
--- a/core/src/main/scala/spark/Cache.scala
+++ b/core/src/main/scala/spark/Cache.scala
@@ -2,9 +2,9 @@ package spark
import java.util.concurrent.atomic.AtomicInteger
-sealed trait CachePutResponse
-case class CachePutSuccess(size: Long) extends CachePutResponse
-case class CachePutFailure() extends CachePutResponse
+private[spark] sealed trait CachePutResponse
+private[spark] case class CachePutSuccess(size: Long) extends CachePutResponse
+private[spark] case class CachePutFailure() extends CachePutResponse
/**
* An interface for caches in Spark, to allow for multiple implementations. Caches are used to store
@@ -22,7 +22,7 @@ case class CachePutFailure() extends CachePutResponse
* This abstract class handles the creation of key spaces, so that subclasses need only deal with
* keys that are unique across modules.
*/
-abstract class Cache {
+private[spark] abstract class Cache {
private val nextKeySpaceId = new AtomicInteger(0)
private def newKeySpaceId() = nextKeySpaceId.getAndIncrement()
@@ -52,7 +52,7 @@ abstract class Cache {
/**
* A key namespace in a Cache.
*/
-class KeySpace(cache: Cache, val keySpaceId: Int) {
+private[spark] class KeySpace(cache: Cache, val keySpaceId: Int) {
def get(datasetId: Any, partition: Int): Any =
cache.get((keySpaceId, datasetId), partition)
diff --git a/core/src/main/scala/spark/CacheTracker.scala b/core/src/main/scala/spark/CacheTracker.scala
index 22110832f8..c5db6ce63a 100644
--- a/core/src/main/scala/spark/CacheTracker.scala
+++ b/core/src/main/scala/spark/CacheTracker.scala
@@ -15,19 +15,20 @@ import scala.collection.mutable.HashSet
import spark.storage.BlockManager
import spark.storage.StorageLevel
-sealed trait CacheTrackerMessage
-case class AddedToCache(rddId: Int, partition: Int, host: String, size: Long = 0L)
+private[spark] sealed trait CacheTrackerMessage
+
+private[spark] case class AddedToCache(rddId: Int, partition: Int, host: String, size: Long = 0L)
extends CacheTrackerMessage
-case class DroppedFromCache(rddId: Int, partition: Int, host: String, size: Long = 0L)
+private[spark] case class DroppedFromCache(rddId: Int, partition: Int, host: String, size: Long = 0L)
extends CacheTrackerMessage
-case class MemoryCacheLost(host: String) extends CacheTrackerMessage
-case class RegisterRDD(rddId: Int, numPartitions: Int) extends CacheTrackerMessage
-case class SlaveCacheStarted(host: String, size: Long) extends CacheTrackerMessage
-case object GetCacheStatus extends CacheTrackerMessage
-case object GetCacheLocations extends CacheTrackerMessage
-case object StopCacheTracker extends CacheTrackerMessage
-
-class CacheTrackerActor extends Actor with Logging {
+private[spark] case class MemoryCacheLost(host: String) extends CacheTrackerMessage
+private[spark] case class RegisterRDD(rddId: Int, numPartitions: Int) extends CacheTrackerMessage
+private[spark] case class SlaveCacheStarted(host: String, size: Long) extends CacheTrackerMessage
+private[spark] case object GetCacheStatus extends CacheTrackerMessage
+private[spark] case object GetCacheLocations extends CacheTrackerMessage
+private[spark] case object StopCacheTracker extends CacheTrackerMessage
+
+private[spark] class CacheTrackerActor extends Actor with Logging {
// TODO: Should probably store (String, CacheType) tuples
private val locs = new HashMap[Int, Array[List[String]]]
@@ -43,8 +44,6 @@ class CacheTrackerActor extends Actor with Logging {
def receive = {
case SlaveCacheStarted(host: String, size: Long) =>
- logInfo("Started slave cache (size %s) on %s".format(
- Utils.memoryBytesToString(size), host))
slaveCapacity.put(host, size)
slaveUsage.put(host, 0)
sender ! true
@@ -56,22 +55,12 @@ class CacheTrackerActor extends Actor with Logging {
case AddedToCache(rddId, partition, host, size) =>
slaveUsage.put(host, getCacheUsage(host) + size)
- logInfo("Cache entry added: (%s, %s) on %s (size added: %s, available: %s)".format(
- rddId, partition, host, Utils.memoryBytesToString(size),
- Utils.memoryBytesToString(getCacheAvailable(host))))
locs(rddId)(partition) = host :: locs(rddId)(partition)
sender ! true
case DroppedFromCache(rddId, partition, host, size) =>
- logInfo("Cache entry removed: (%s, %s) on %s (size dropped: %s, available: %s)".format(
- rddId, partition, host, Utils.memoryBytesToString(size),
- Utils.memoryBytesToString(getCacheAvailable(host))))
slaveUsage.put(host, getCacheUsage(host) - size)
// Do a sanity check to make sure usage is greater than 0.
- val usage = getCacheUsage(host)
- if (usage < 0) {
- logError("Cache usage on %s is negative (%d)".format(host, usage))
- }
locs(rddId)(partition) = locs(rddId)(partition).filterNot(_ == host)
sender ! true
@@ -101,7 +90,7 @@ class CacheTrackerActor extends Actor with Logging {
}
}
-class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: BlockManager)
+private[spark] class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: BlockManager)
extends Logging {
// Tracker actor on the master, or remote reference to it on workers
@@ -151,7 +140,6 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl
logInfo("Registering RDD ID " + rddId + " with cache")
registeredRddIds += rddId
communicate(RegisterRDD(rddId, numPartitions))
- logInfo(RegisterRDD(rddId, numPartitions) + " successful")
}
}
}
@@ -169,9 +157,8 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl
}
// For BlockManager.scala only
- def notifyTheCacheTrackerFromBlockManager(t: AddedToCache) {
+ def notifyFromBlockManager(t: AddedToCache) {
communicate(t)
- logInfo("notifyTheCacheTrackerFromBlockManager successful")
}
// Get a snapshot of the currently known locations
@@ -181,7 +168,7 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl
// Gets or computes an RDD split
def getOrCompute[T](rdd: RDD[T], split: Split, storageLevel: StorageLevel): Iterator[T] = {
- val key = "rdd:%d:%d".format(rdd.id, split.index)
+ val key = "rdd_%d_%d".format(rdd.id, split.index)
logInfo("Cache key is " + key)
blockManager.get(key) match {
case Some(cachedValues) =>
@@ -221,23 +208,19 @@ class CacheTracker(actorSystem: ActorSystem, isMaster: Boolean, blockManager: Bl
// TODO: fetch any remote copy of the split that may be available
// TODO: also register a listener for when it unloads
logInfo("Computing partition " + split)
+ val elements = new ArrayBuffer[Any]
+ elements ++= rdd.compute(split)
try {
- // BlockManager will iterate over results from compute to create RDD
- blockManager.put(key, rdd.compute(split), storageLevel, false)
+ // Try to put this block in the blockManager
+ blockManager.put(key, elements, storageLevel, true)
//future.apply() // Wait for the reply from the cache tracker
- blockManager.get(key) match {
- case Some(values) =>
- return values.asInstanceOf[Iterator[T]]
- case None =>
- logWarning("loading partition failed after computing it " + key)
- return null
- }
} finally {
loading.synchronized {
loading.remove(key)
loading.notifyAll()
}
}
+ return elements.iterator.asInstanceOf[Iterator[T]]
}
}
diff --git a/core/src/main/scala/spark/ClosureCleaner.scala b/core/src/main/scala/spark/ClosureCleaner.scala
index 3b83d23a13..98525b99c8 100644
--- a/core/src/main/scala/spark/ClosureCleaner.scala
+++ b/core/src/main/scala/spark/ClosureCleaner.scala
@@ -9,7 +9,7 @@ import org.objectweb.asm.{ClassReader, MethodVisitor, Type}
import org.objectweb.asm.commons.EmptyVisitor
import org.objectweb.asm.Opcodes._
-object ClosureCleaner extends Logging {
+private[spark] object ClosureCleaner extends Logging {
// Get an ASM class reader for a given class from the JAR that loaded it
private def getClassReader(cls: Class[_]): ClassReader = {
new ClassReader(cls.getResourceAsStream(
@@ -154,7 +154,7 @@ object ClosureCleaner extends Logging {
}
}
-class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor {
+private[spark] class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor {
override def visitMethod(access: Int, name: String, desc: String,
sig: String, exceptions: Array[String]): MethodVisitor = {
return new EmptyVisitor {
@@ -180,7 +180,7 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends EmptyVisitor
}
}
-class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor {
+private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends EmptyVisitor {
var myName: String = null
override def visit(version: Int, access: Int, name: String, sig: String,
diff --git a/core/src/main/scala/spark/Dependency.scala b/core/src/main/scala/spark/Dependency.scala
index c0ff94acc6..b85d2732db 100644
--- a/core/src/main/scala/spark/Dependency.scala
+++ b/core/src/main/scala/spark/Dependency.scala
@@ -1,22 +1,51 @@
package spark
-abstract class Dependency[T](val rdd: RDD[T], val isShuffle: Boolean) extends Serializable
+/**
+ * Base class for dependencies.
+ */
+abstract class Dependency[T](val rdd: RDD[T]) extends Serializable
-abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd, false) {
+/**
+ * Base class for dependencies where each partition of the parent RDD is used by at most one
+ * partition of the child RDD. Narrow dependencies allow for pipelined execution.
+ */
+abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
+ /**
+ * Get the parent partitions for a child partition.
+ * @param outputPartition a partition of the child RDD
+ * @return the partitions of the parent RDD that the child partition depends upon
+ */
def getParents(outputPartition: Int): Seq[Int]
}
-class ShuffleDependency[K, V, C](
- val shuffleId: Int,
+/**
+ * Represents a dependency on the output of a shuffle stage.
+ * @param shuffleId the shuffle id
+ * @param rdd the parent RDD
+ * @param partitioner partitioner used to partition the shuffle output
+ */
+class ShuffleDependency[K, V](
@transient rdd: RDD[(K, V)],
- val aggregator: Aggregator[K, V, C],
val partitioner: Partitioner)
- extends Dependency(rdd, true)
+ extends Dependency(rdd) {
+ val shuffleId: Int = rdd.context.newShuffleId()
+}
+
+/**
+ * Represents a one-to-one dependency between partitions of the parent and child RDDs.
+ */
class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) {
override def getParents(partitionId: Int) = List(partitionId)
}
+/**
+ * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs.
+ * @param rdd the parent RDD
+ * @param inStart the start of the range in the parent RDD
+ * @param outStart the start of the range in the child RDD
+ * @param length the length of the range
+ */
class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int)
extends NarrowDependency[T](rdd) {
diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala
index 1fbf66b7de..b2a0e2b631 100644
--- a/core/src/main/scala/spark/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala
@@ -4,33 +4,49 @@ import spark.partial.BoundedDouble
import spark.partial.MeanEvaluator
import spark.partial.PartialResult
import spark.partial.SumEvaluator
-
import spark.util.StatCounter
/**
* Extra functions available on RDDs of Doubles through an implicit conversion.
+ * Import `spark.SparkContext._` at the top of your program to use these functions.
*/
class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
+ /** Add up the elements in this RDD. */
def sum(): Double = {
self.reduce(_ + _)
}
+ /**
+ * Return a [[spark.util.StatCounter]] object that captures the mean, variance and count
+ * of the RDD's elements in one operation.
+ */
def stats(): StatCounter = {
self.mapPartitions(nums => Iterator(StatCounter(nums))).reduce((a, b) => a.merge(b))
}
+ /** Compute the mean of this RDD's elements. */
def mean(): Double = stats().mean
+ /** Compute the variance of this RDD's elements. */
def variance(): Double = stats().variance
+ /** Compute the standard deviation of this RDD's elements. */
def stdev(): Double = stats().stdev
+ /**
+ * Compute the sample standard deviation of this RDD's elements (which corrects for bias in
+ * estimating the standard deviation by dividing by N-1 instead of N).
+ */
+ def sampleStdev(): Double = stats().stdev
+
+ /** (Experimental) Approximate operation to return the mean within a timeout. */
def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
val evaluator = new MeanEvaluator(self.splits.size, confidence)
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
}
+ /** (Experimental) Approximate operation to return the sum within a timeout. */
def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
val evaluator = new SumEvaluator(self.splits.size, confidence)
diff --git a/core/src/main/scala/spark/FetchFailedException.scala b/core/src/main/scala/spark/FetchFailedException.scala
index 55512f4481..a953081d24 100644
--- a/core/src/main/scala/spark/FetchFailedException.scala
+++ b/core/src/main/scala/spark/FetchFailedException.scala
@@ -2,7 +2,7 @@ package spark
import spark.storage.BlockManagerId
-class FetchFailedException(
+private[spark] class FetchFailedException(
val bmAddress: BlockManagerId,
val shuffleId: Int,
val mapId: Int,
diff --git a/core/src/main/scala/spark/HadoopWriter.scala b/core/src/main/scala/spark/HadoopWriter.scala
index 12b6a0954c..afcf9f6db4 100644
--- a/core/src/main/scala/spark/HadoopWriter.scala
+++ b/core/src/main/scala/spark/HadoopWriter.scala
@@ -16,11 +16,14 @@ import spark.Logging
import spark.SerializableWritable
/**
- * Saves an RDD using a Hadoop OutputFormat as specified by a JobConf. The JobConf should also
- * contain an output key class, an output value class, a filename to write to, etc exactly like in
- * a Hadoop job.
+ * Internal helper class that saves an RDD using a Hadoop OutputFormat. This is only public
+ * because we need to access this class from the `spark` package to use some package-private Hadoop
+ * functions, but this class should not be used directly by users.
+ *
+ * Saves the RDD using a JobConf, which should contain an output key class, an output value class,
+ * a filename to write to, etc, exactly like in a Hadoop MapReduce job.
*/
-class HadoopWriter(@transient jobConf: JobConf) extends Logging with Serializable {
+class HadoopWriter(@transient jobConf: JobConf) extends Logging with HadoopMapRedUtil with Serializable {
private val now = new Date()
private val conf = new SerializableWritable(jobConf)
@@ -42,7 +45,7 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with Serializabl
setConfParams()
val jCtxt = getJobContext()
- getOutputCommitter().setupJob(jCtxt)
+ getOutputCommitter().setupJob(jCtxt)
}
@@ -126,14 +129,14 @@ class HadoopWriter(@transient jobConf: JobConf) extends Logging with Serializabl
private def getJobContext(): JobContext = {
if (jobContext == null) {
- jobContext = new JobContext(conf.value, jID.value)
+ jobContext = newJobContext(conf.value, jID.value)
}
return jobContext
}
private def getTaskContext(): TaskAttemptContext = {
if (taskContext == null) {
- taskContext = new TaskAttemptContext(conf.value, taID.value)
+ taskContext = newTaskAttemptContext(conf.value, taID.value)
}
return taskContext
}
diff --git a/core/src/main/scala/spark/HttpFileServer.scala b/core/src/main/scala/spark/HttpFileServer.scala
new file mode 100644
index 0000000000..659d17718f
--- /dev/null
+++ b/core/src/main/scala/spark/HttpFileServer.scala
@@ -0,0 +1,47 @@
+package spark
+
+import java.io.{File, PrintWriter}
+import java.net.URL
+import scala.collection.mutable.HashMap
+import org.apache.hadoop.fs.FileUtil
+
+private[spark] class HttpFileServer extends Logging {
+
+ var baseDir : File = null
+ var fileDir : File = null
+ var jarDir : File = null
+ var httpServer : HttpServer = null
+ var serverUri : String = null
+
+ def initialize() {
+ baseDir = Utils.createTempDir()
+ fileDir = new File(baseDir, "files")
+ jarDir = new File(baseDir, "jars")
+ fileDir.mkdir()
+ jarDir.mkdir()
+ logInfo("HTTP File server directory is " + baseDir)
+ httpServer = new HttpServer(baseDir)
+ httpServer.start()
+ serverUri = httpServer.uri
+ }
+
+ def stop() {
+ httpServer.stop()
+ }
+
+ def addFile(file: File) : String = {
+ addFileToDir(file, fileDir)
+ return serverUri + "/files/" + file.getName
+ }
+
+ def addJar(file: File) : String = {
+ addFileToDir(file, jarDir)
+ return serverUri + "/jars/" + file.getName
+ }
+
+ def addFileToDir(file: File, dir: File) : String = {
+ Utils.copyFile(file, new File(dir, file.getName))
+ return dir + "/" + file.getName
+ }
+
+} \ No newline at end of file
diff --git a/core/src/main/scala/spark/HttpServer.scala b/core/src/main/scala/spark/HttpServer.scala
index 855f2c752f..0196595ba1 100644
--- a/core/src/main/scala/spark/HttpServer.scala
+++ b/core/src/main/scala/spark/HttpServer.scala
@@ -12,14 +12,14 @@ import org.eclipse.jetty.util.thread.QueuedThreadPool
/**
* Exception type thrown by HttpServer when it is in the wrong state for an operation.
*/
-class ServerStateException(message: String) extends Exception(message)
+private[spark] class ServerStateException(message: String) extends Exception(message)
/**
* An HTTP server for static content used to allow worker nodes to access JARs added to SparkContext
* as well as classes created by the interpreter when the user types in code. This is just a wrapper
* around a Jetty server.
*/
-class HttpServer(resourceBase: File) extends Logging {
+private[spark] class HttpServer(resourceBase: File) extends Logging {
private var server: Server = null
private var port: Int = -1
diff --git a/core/src/main/scala/spark/JavaSerializer.scala b/core/src/main/scala/spark/JavaSerializer.scala
index d11ba5167d..b04a27d073 100644
--- a/core/src/main/scala/spark/JavaSerializer.scala
+++ b/core/src/main/scala/spark/JavaSerializer.scala
@@ -3,16 +3,17 @@ package spark
import java.io._
import java.nio.ByteBuffer
+import serializer.{Serializer, SerializerInstance, DeserializationStream, SerializationStream}
import spark.util.ByteBufferInputStream
-class JavaSerializationStream(out: OutputStream) extends SerializationStream {
+private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream {
val objOut = new ObjectOutputStream(out)
- def writeObject[T](t: T) { objOut.writeObject(t) }
+ def writeObject[T](t: T): SerializationStream = { objOut.writeObject(t); this }
def flush() { objOut.flush() }
def close() { objOut.close() }
}
-class JavaDeserializationStream(in: InputStream, loader: ClassLoader)
+private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader)
extends DeserializationStream {
val objIn = new ObjectInputStream(in) {
override def resolveClass(desc: ObjectStreamClass) =
@@ -23,7 +24,7 @@ extends DeserializationStream {
def close() { objIn.close() }
}
-class JavaSerializerInstance extends SerializerInstance {
+private[spark] class JavaSerializerInstance extends SerializerInstance {
def serialize[T](t: T): ByteBuffer = {
val bos = new ByteArrayOutputStream()
val out = serializeStream(bos)
@@ -57,6 +58,9 @@ class JavaSerializerInstance extends SerializerInstance {
}
}
+/**
+ * A Spark serializer that uses Java's built-in serialization.
+ */
class JavaSerializer extends Serializer {
def newInstance(): SerializerInstance = new JavaSerializerInstance
}
diff --git a/core/src/main/scala/spark/KryoSerializer.scala b/core/src/main/scala/spark/KryoSerializer.scala
index 8a3f565071..44b630e478 100644
--- a/core/src/main/scala/spark/KryoSerializer.scala
+++ b/core/src/main/scala/spark/KryoSerializer.scala
@@ -13,6 +13,7 @@ import com.esotericsoftware.kryo.serialize.ClassSerializer
import com.esotericsoftware.kryo.serialize.SerializableSerializer
import de.javakaffee.kryoserializers.KryoReflectionFactorySupport
+import serializer.{SerializerInstance, DeserializationStream, SerializationStream}
import spark.broadcast._
import spark.storage._
@@ -20,7 +21,7 @@ import spark.storage._
* Zig-zag encoder used to write object sizes to serialization streams.
* Based on Kryo's integer encoder.
*/
-object ZigZag {
+private[spark] object ZigZag {
def writeInt(n: Int, out: OutputStream) {
var value = n
if ((value & ~0x7F) == 0) {
@@ -68,22 +69,25 @@ object ZigZag {
}
}
+private[spark]
class KryoSerializationStream(kryo: Kryo, threadBuffer: ByteBuffer, out: OutputStream)
extends SerializationStream {
val channel = Channels.newChannel(out)
- def writeObject[T](t: T) {
+ def writeObject[T](t: T): SerializationStream = {
kryo.writeClassAndObject(threadBuffer, t)
ZigZag.writeInt(threadBuffer.position(), out)
threadBuffer.flip()
channel.write(threadBuffer)
threadBuffer.clear()
+ this
}
def flush() { out.flush() }
def close() { out.close() }
}
+private[spark]
class KryoDeserializationStream(objectBuffer: ObjectBuffer, in: InputStream)
extends DeserializationStream {
def readObject[T](): T = {
@@ -94,7 +98,7 @@ extends DeserializationStream {
def close() { in.close() }
}
-class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance {
+private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance {
val kryo = ks.kryo
val threadBuffer = ks.threadBuffer.get()
val objectBuffer = ks.objectBuffer.get()
@@ -155,13 +159,21 @@ class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance {
}
}
-// Used by clients to register their own classes
+/**
+ * Interface implemented by clients to register their classes with Kryo when using Kryo
+ * serialization.
+ */
trait KryoRegistrator {
def registerClasses(kryo: Kryo): Unit
}
-class KryoSerializer extends Serializer with Logging {
- val kryo = createKryo()
+/**
+ * A Spark serializer that uses the [[http://code.google.com/p/kryo/wiki/V1Documentation Kryo 1.x library]].
+ */
+class KryoSerializer extends spark.serializer.Serializer with Logging {
+ // Make this lazy so that it only gets called once we receive our first task on each executor,
+ // so we can pull out any custom Kryo registrator from the user's JARs.
+ lazy val kryo = createKryo()
val bufferSize = System.getProperty("spark.kryoserializer.buffer.mb", "32").toInt * 1024 * 1024
@@ -192,8 +204,8 @@ class KryoSerializer extends Serializer with Logging {
(1, 1, 1), (1, 1, 1, 1), (1, 1, 1, 1, 1),
None,
ByteBuffer.allocate(1),
- StorageLevel.MEMORY_ONLY_DESER,
- PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY_DESER),
+ StorageLevel.MEMORY_ONLY,
+ PutBlock("1", ByteBuffer.allocate(1), StorageLevel.MEMORY_ONLY),
GotBlock("1", ByteBuffer.allocate(1)),
GetBlock("1")
)
@@ -256,7 +268,8 @@ class KryoSerializer extends Serializer with Logging {
val regCls = System.getProperty("spark.kryo.registrator")
if (regCls != null) {
logInfo("Running user registrator: " + regCls)
- val reg = Class.forName(regCls).newInstance().asInstanceOf[KryoRegistrator]
+ val classLoader = Thread.currentThread.getContextClassLoader
+ val reg = Class.forName(regCls, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]
reg.registerClasses(kryo)
}
kryo
diff --git a/core/src/main/scala/spark/Logging.scala b/core/src/main/scala/spark/Logging.scala
index 69935b86de..90bae26202 100644
--- a/core/src/main/scala/spark/Logging.scala
+++ b/core/src/main/scala/spark/Logging.scala
@@ -15,7 +15,7 @@ trait Logging {
private var log_ : Logger = null
// Method to get or create the logger for this object
- def log: Logger = {
+ protected def log: Logger = {
if (log_ == null) {
var className = this.getClass.getName
// Ignore trailing $'s in the class names for Scala objects
@@ -28,48 +28,48 @@ trait Logging {
}
// Log methods that take only a String
- def logInfo(msg: => String) {
+ protected def logInfo(msg: => String) {
if (log.isInfoEnabled) log.info(msg)
}
- def logDebug(msg: => String) {
+ protected def logDebug(msg: => String) {
if (log.isDebugEnabled) log.debug(msg)
}
- def logTrace(msg: => String) {
+ protected def logTrace(msg: => String) {
if (log.isTraceEnabled) log.trace(msg)
}
- def logWarning(msg: => String) {
+ protected def logWarning(msg: => String) {
if (log.isWarnEnabled) log.warn(msg)
}
- def logError(msg: => String) {
+ protected def logError(msg: => String) {
if (log.isErrorEnabled) log.error(msg)
}
// Log methods that take Throwables (Exceptions/Errors) too
- def logInfo(msg: => String, throwable: Throwable) {
+ protected def logInfo(msg: => String, throwable: Throwable) {
if (log.isInfoEnabled) log.info(msg, throwable)
}
- def logDebug(msg: => String, throwable: Throwable) {
+ protected def logDebug(msg: => String, throwable: Throwable) {
if (log.isDebugEnabled) log.debug(msg, throwable)
}
- def logTrace(msg: => String, throwable: Throwable) {
+ protected def logTrace(msg: => String, throwable: Throwable) {
if (log.isTraceEnabled) log.trace(msg, throwable)
}
- def logWarning(msg: => String, throwable: Throwable) {
+ protected def logWarning(msg: => String, throwable: Throwable) {
if (log.isWarnEnabled) log.warn(msg, throwable)
}
- def logError(msg: => String, throwable: Throwable) {
+ protected def logError(msg: => String, throwable: Throwable) {
if (log.isErrorEnabled) log.error(msg, throwable)
}
// Method for ensuring that logging is initialized, to avoid having multiple
// threads do it concurrently (as SLF4J initialization is not thread safe).
- def initLogging() { log }
+ protected def initLogging() { log }
}
diff --git a/core/src/main/scala/spark/MapOutputTracker.scala b/core/src/main/scala/spark/MapOutputTracker.scala
index 82c1391345..45441aa5e5 100644
--- a/core/src/main/scala/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/spark/MapOutputTracker.scala
@@ -1,6 +1,6 @@
package spark
-import java.io.{DataInputStream, DataOutputStream, ByteArrayOutputStream, ByteArrayInputStream}
+import java.io._
import java.util.concurrent.ConcurrentHashMap
import akka.actor._
@@ -14,16 +14,19 @@ import akka.util.duration._
import scala.collection.mutable.HashMap
import scala.collection.mutable.HashSet
+import scheduler.MapStatus
import spark.storage.BlockManagerId
+import java.util.zip.{GZIPInputStream, GZIPOutputStream}
-sealed trait MapOutputTrackerMessage
-case class GetMapOutputLocations(shuffleId: Int) extends MapOutputTrackerMessage
-case object StopMapOutputTracker extends MapOutputTrackerMessage
+private[spark] sealed trait MapOutputTrackerMessage
+private[spark] case class GetMapOutputStatuses(shuffleId: Int, requester: String)
+ extends MapOutputTrackerMessage
+private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage
-class MapOutputTrackerActor(tracker: MapOutputTracker) extends Actor with Logging {
+private[spark] class MapOutputTrackerActor(tracker: MapOutputTracker) extends Actor with Logging {
def receive = {
- case GetMapOutputLocations(shuffleId: Int) =>
- logInfo("Asked to get map output locations for shuffle " + shuffleId)
+ case GetMapOutputStatuses(shuffleId: Int, requester: String) =>
+ logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + requester)
sender ! tracker.getSerializedLocations(shuffleId)
case StopMapOutputTracker =>
@@ -33,23 +36,23 @@ class MapOutputTrackerActor(tracker: MapOutputTracker) extends Actor with Loggin
}
}
-class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logging {
+private[spark] class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logging {
val ip: String = System.getProperty("spark.master.host", "localhost")
val port: Int = System.getProperty("spark.master.port", "7077").toInt
val actorName: String = "MapOutputTracker"
val timeout = 10.seconds
- var bmAddresses = new ConcurrentHashMap[Int, Array[BlockManagerId]]
+ var mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
// Incremented every time a fetch fails so that client nodes know to clear
// their cache of map output locations if this happens.
private var generation: Long = 0
- private var generationLock = new java.lang.Object
+ private val generationLock = new java.lang.Object
- // Cache a serialized version of the output locations for each shuffle to send them out faster
+ // Cache a serialized version of the output statuses for each shuffle to send them out faster
var cacheGeneration = generation
- val cachedSerializedLocs = new HashMap[Int, Array[Byte]]
+ val cachedSerializedStatuses = new HashMap[Int, Array[Byte]]
var trackerActor: ActorRef = if (isMaster) {
val actor = actorSystem.actorOf(Props(new MapOutputTrackerActor(this)), name = actorName)
@@ -80,31 +83,34 @@ class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logg
}
def registerShuffle(shuffleId: Int, numMaps: Int) {
- if (bmAddresses.get(shuffleId) != null) {
+ if (mapStatuses.get(shuffleId) != null) {
throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice")
}
- bmAddresses.put(shuffleId, new Array[BlockManagerId](numMaps))
+ mapStatuses.put(shuffleId, new Array[MapStatus](numMaps))
}
- def registerMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
- var array = bmAddresses.get(shuffleId)
+ def registerMapOutput(shuffleId: Int, mapId: Int, status: MapStatus) {
+ var array = mapStatuses.get(shuffleId)
array.synchronized {
- array(mapId) = bmAddress
+ array(mapId) = status
}
}
- def registerMapOutputs(shuffleId: Int, locs: Array[BlockManagerId], changeGeneration: Boolean = false) {
- bmAddresses.put(shuffleId, Array[BlockManagerId]() ++ locs)
+ def registerMapOutputs(
+ shuffleId: Int,
+ statuses: Array[MapStatus],
+ changeGeneration: Boolean = false) {
+ mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses)
if (changeGeneration) {
incrementGeneration()
}
}
def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) {
- var array = bmAddresses.get(shuffleId)
+ var array = mapStatuses.get(shuffleId)
if (array != null) {
array.synchronized {
- if (array(mapId) == bmAddress) {
+ if (array(mapId).address == bmAddress) {
array(mapId) = null
}
}
@@ -117,10 +123,10 @@ class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logg
// Remembers which map output locations are currently being fetched on a worker
val fetching = new HashSet[Int]
- // Called on possibly remote nodes to get the server URIs for a given shuffle
- def getServerAddresses(shuffleId: Int): Array[BlockManagerId] = {
- val locs = bmAddresses.get(shuffleId)
- if (locs == null) {
+ // Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle
+ def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = {
+ val statuses = mapStatuses.get(shuffleId)
+ if (statuses == null) {
logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them")
fetching.synchronized {
if (fetching.contains(shuffleId)) {
@@ -129,34 +135,38 @@ class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logg
try {
fetching.wait()
} catch {
- case _ =>
+ case e: InterruptedException =>
}
}
- return bmAddresses.get(shuffleId)
+ return mapStatuses.get(shuffleId).map(status =>
+ (status.address, MapOutputTracker.decompressSize(status.compressedSizes(reduceId))))
} else {
fetching += shuffleId
}
}
// We won the race to fetch the output locs; do so
logInfo("Doing the fetch; tracker actor = " + trackerActor)
- val fetchedBytes = askTracker(GetMapOutputLocations(shuffleId)).asInstanceOf[Array[Byte]]
- val fetchedLocs = deserializeLocations(fetchedBytes)
+ val host = System.getProperty("spark.hostname", Utils.localHostName)
+ val fetchedBytes = askTracker(GetMapOutputStatuses(shuffleId, host)).asInstanceOf[Array[Byte]]
+ val fetchedStatuses = deserializeStatuses(fetchedBytes)
logInfo("Got the output locations")
- bmAddresses.put(shuffleId, fetchedLocs)
+ mapStatuses.put(shuffleId, fetchedStatuses)
fetching.synchronized {
fetching -= shuffleId
fetching.notifyAll()
}
- return fetchedLocs
+ return fetchedStatuses.map(s =>
+ (s.address, MapOutputTracker.decompressSize(s.compressedSizes(reduceId))))
} else {
- return locs
+ return statuses.map(s =>
+ (s.address, MapOutputTracker.decompressSize(s.compressedSizes(reduceId))))
}
}
def stop() {
communicate(StopMapOutputTracker)
- bmAddresses.clear()
+ mapStatuses.clear()
trackerActor = null
}
@@ -182,75 +192,83 @@ class MapOutputTracker(actorSystem: ActorSystem, isMaster: Boolean) extends Logg
generationLock.synchronized {
if (newGen > generation) {
logInfo("Updating generation to " + newGen + " and clearing cache")
- bmAddresses = new ConcurrentHashMap[Int, Array[BlockManagerId]]
+ mapStatuses = new ConcurrentHashMap[Int, Array[MapStatus]]
generation = newGen
}
}
}
def getSerializedLocations(shuffleId: Int): Array[Byte] = {
- var locs: Array[BlockManagerId] = null
+ var statuses: Array[MapStatus] = null
var generationGotten: Long = -1
generationLock.synchronized {
if (generation > cacheGeneration) {
- cachedSerializedLocs.clear()
+ cachedSerializedStatuses.clear()
cacheGeneration = generation
}
- cachedSerializedLocs.get(shuffleId) match {
+ cachedSerializedStatuses.get(shuffleId) match {
case Some(bytes) =>
return bytes
case None =>
- locs = bmAddresses.get(shuffleId)
+ statuses = mapStatuses.get(shuffleId)
generationGotten = generation
}
}
// 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
- val bytes = serializeLocations(locs)
+ val bytes = serializeStatuses(statuses)
+ logInfo("Size of output statuses for shuffle %d is %d bytes".format(shuffleId, bytes.length))
// Add them into the table only if the generation hasn't changed while we were working
generationLock.synchronized {
if (generation == generationGotten) {
- cachedSerializedLocs(shuffleId) = bytes
+ cachedSerializedStatuses(shuffleId) = bytes
}
}
return bytes
}
// Serialize an array of map output locations into an efficient byte format so that we can send
- // it to reduce tasks. We do this by grouping together the locations by block manager ID.
- def serializeLocations(locs: Array[BlockManagerId]): Array[Byte] = {
+ // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will
+ // generally be pretty compressible because many map outputs will be on the same hostname.
+ def serializeStatuses(statuses: Array[MapStatus]): Array[Byte] = {
val out = new ByteArrayOutputStream
- val dataOut = new DataOutputStream(out)
- dataOut.writeInt(locs.length)
- val grouped = locs.zipWithIndex.groupBy(_._1)
- dataOut.writeInt(grouped.size)
- for ((id, pairs) <- grouped if id != null) {
- dataOut.writeUTF(id.ip)
- dataOut.writeInt(id.port)
- dataOut.writeInt(pairs.length)
- for ((_, blockIndex) <- pairs) {
- dataOut.writeInt(blockIndex)
- }
- }
- dataOut.close()
+ val objOut = new ObjectOutputStream(new GZIPOutputStream(out))
+ objOut.writeObject(statuses)
+ objOut.close()
out.toByteArray
}
- // Opposite of serializeLocations.
- def deserializeLocations(bytes: Array[Byte]): Array[BlockManagerId] = {
- val dataIn = new DataInputStream(new ByteArrayInputStream(bytes))
- val length = dataIn.readInt()
- val array = new Array[BlockManagerId](length)
- val numGroups = dataIn.readInt()
- for (i <- 0 until numGroups) {
- val ip = dataIn.readUTF()
- val port = dataIn.readInt()
- val id = new BlockManagerId(ip, port)
- val numBlocks = dataIn.readInt()
- for (j <- 0 until numBlocks) {
- array(dataIn.readInt()) = id
- }
+ // Opposite of serializeStatuses.
+ def deserializeStatuses(bytes: Array[Byte]): Array[MapStatus] = {
+ val objIn = new ObjectInputStream(new GZIPInputStream(new ByteArrayInputStream(bytes)))
+ objIn.readObject().asInstanceOf[Array[MapStatus]]
+ }
+}
+
+private[spark] object MapOutputTracker {
+ private val LOG_BASE = 1.1
+
+ /**
+ * Compress a size in bytes to 8 bits for efficient reporting of map output sizes.
+ * We do this by encoding the log base 1.1 of the size as an integer, which can support
+ * sizes up to 35 GB with at most 10% error.
+ */
+ def compressSize(size: Long): Byte = {
+ if (size <= 1L) {
+ 0
+ } else {
+ math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte
+ }
+ }
+
+ /**
+ * Decompress an 8-bit encoded block size, using the reverse operation of compressSize.
+ */
+ def decompressSize(compressedSize: Byte): Long = {
+ if (compressedSize == 0) {
+ 1
+ } else {
+ math.pow(LOG_BASE, (compressedSize & 0xFF)).toLong
}
- array
}
}
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index 64018f8c6b..e5bb639cfd 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -1,11 +1,6 @@
package spark
-import java.io.EOFException
-import java.net.URL
-import java.io.ObjectInputStream
-import java.util.concurrent.atomic.AtomicLong
-import java.util.{HashMap => JHashMap}
-import java.util.Date
+import java.util.{Date, HashMap => JHashMap}
import java.text.SimpleDateFormat
import scala.collection.Map
@@ -15,46 +10,66 @@ import scala.collection.JavaConversions._
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
-import org.apache.hadoop.io.BytesWritable
-import org.apache.hadoop.io.NullWritable
-import org.apache.hadoop.io.Text
-import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapred.FileOutputCommitter
import org.apache.hadoop.mapred.FileOutputFormat
import org.apache.hadoop.mapred.HadoopWriter
import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapred.OutputCommitter
import org.apache.hadoop.mapred.OutputFormat
-import org.apache.hadoop.mapred.SequenceFileOutputFormat
-import org.apache.hadoop.mapred.TextOutputFormat
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
-import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
-import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter}
-import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob}
-import org.apache.hadoop.mapreduce.TaskAttemptID
-import org.apache.hadoop.mapreduce.TaskAttemptContext
+import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, Job => NewAPIHadoopJob, HadoopMapReduceUtil, TaskAttemptID, TaskAttemptContext}
-import spark.SparkContext._
import spark.partial.BoundedDouble
import spark.partial.PartialResult
+import spark.rdd._
+import spark.SparkContext._
/**
* Extra functions available on RDDs of (key, value) pairs through an implicit conversion.
+ * Import `spark.SparkContext._` at the top of your program to use these functions.
*/
class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
self: RDD[(K, V)])
extends Logging
+ with HadoopMapReduceUtil
with Serializable {
+ /**
+ * Generic function to combine the elements for each key using a custom set of aggregation
+ * functions. Turns an RDD[(K, V)] into a result of type RDD[(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, Seq[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.
+ *
+ * In addition, users can control the partitioning of the output RDD, and whether to perform
+ * map-side aggregation (if a mapper can produce multiple items with the same key).
+ */
def combineByKey[C](createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C,
- partitioner: Partitioner): RDD[(K, C)] = {
- val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
- new ShuffledRDD(self, aggregator, partitioner)
+ partitioner: Partitioner,
+ mapSideCombine: Boolean = true): RDD[(K, C)] = {
+ val aggregator =
+ new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
+ if (mapSideCombine) {
+ val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true)
+ val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner)
+ partitioned.mapPartitions(aggregator.combineCombinersByKey(_), true)
+ } else {
+ // Don't apply map-side combiner.
+ // A sanity check to make sure mergeCombiners is not defined.
+ assert(mergeCombiners == null)
+ val values = new ShuffledRDD[K, V](self, partitioner)
+ values.mapPartitions(aggregator.combineValuesByKey(_), true)
+ }
}
+ /**
+ * Simplified version of combineByKey that hash-partitions the output RDD.
+ */
def combineByKey[C](createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C,
@@ -62,10 +77,20 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numSplits))
}
+ /**
+ * 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.
+ */
def reduceByKey(partitioner: Partitioner, func: (V, V) => V): RDD[(K, V)] = {
combineByKey[V]((v: V) => v, func, func, partitioner)
}
-
+
+ /**
+ * Merge the values for each key using an associative reduce function, but return the results
+ * immediately to the master as a Map. This will also perform the merging locally on each mapper
+ * before sending results to a reducer, similarly to a "combiner" in MapReduce.
+ */
def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = {
def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = {
val map = new JHashMap[K, V]
@@ -87,22 +112,34 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
self.mapPartitions(reducePartition).reduce(mergeMaps)
}
- // Alias for backwards compatibility
+ /** Alias for reduceByKeyLocally */
def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = reduceByKeyLocally(func)
- // TODO: This should probably be a distributed version
+ /** Count the number of elements for each key, and return the result to the master as a Map. */
def countByKey(): Map[K, Long] = self.map(_._1).countByValue()
- // TODO: This should probably be a distributed version
+ /**
+ * (Experimental) Approximate version of countByKey that can return a partial result if it does
+ * not finish within a timeout.
+ */
def countByKeyApprox(timeout: Long, confidence: Double = 0.95)
: PartialResult[Map[K, BoundedDouble]] = {
self.map(_._1).countByValueApprox(timeout, confidence)
}
+ /**
+ * 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 numSplits splits.
+ */
def reduceByKey(func: (V, V) => V, numSplits: Int): RDD[(K, V)] = {
reduceByKey(new HashPartitioner(numSplits), func)
}
+ /**
+ * Group the values for each key in the RDD into a single sequence. Allows controlling the
+ * partitioning of the resulting key-value pair RDD by passing a Partitioner.
+ */
def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = {
def createCombiner(v: V) = ArrayBuffer(v)
def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v
@@ -112,19 +149,39 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
bufs.asInstanceOf[RDD[(K, Seq[V])]]
}
+ /**
+ * Group the values for each key in the RDD into a single sequence. Hash-partitions the
+ * resulting RDD with into `numSplits` partitions.
+ */
def groupByKey(numSplits: Int): RDD[(K, Seq[V])] = {
groupByKey(new HashPartitioner(numSplits))
}
- def partitionBy(partitioner: Partitioner): RDD[(K, V)] = {
- def createCombiner(v: V) = ArrayBuffer(v)
- def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v
- def mergeCombiners(b1: ArrayBuffer[V], b2: ArrayBuffer[V]) = b1 ++= b2
- val bufs = combineByKey[ArrayBuffer[V]](
- createCombiner _, mergeValue _, mergeCombiners _, partitioner)
- bufs.flatMapValues(buf => buf)
+ /**
+ * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine`
+ * is true, Spark will group values of the same key together on the map side before the
+ * repartitioning, to only send each key over the network once. If a large number of
+ * duplicated keys are expected, and the size of the keys are large, `mapSideCombine` should
+ * be set to true.
+ */
+ def partitionBy(partitioner: Partitioner, mapSideCombine: Boolean = false): RDD[(K, V)] = {
+ if (mapSideCombine) {
+ def createCombiner(v: V) = ArrayBuffer(v)
+ def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v
+ def mergeCombiners(b1: ArrayBuffer[V], b2: ArrayBuffer[V]) = b1 ++= b2
+ val bufs = combineByKey[ArrayBuffer[V]](
+ createCombiner _, mergeValue _, mergeCombiners _, partitioner)
+ bufs.flatMapValues(buf => buf)
+ } else {
+ new ShuffledRDD[K, V](self, partitioner)
+ }
}
+ /**
+ * 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.
+ */
def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = {
this.cogroup(other, partitioner).flatMapValues {
case (vs, ws) =>
@@ -132,6 +189,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
}
+ /**
+ * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
+ * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
+ * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to
+ * partition the output RDD.
+ */
def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = {
this.cogroup(other, partitioner).flatMapValues {
case (vs, ws) =>
@@ -143,6 +206,12 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
}
+ /**
+ * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
+ * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
+ * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to
+ * partition the output RDD.
+ */
def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner)
: RDD[(K, (Option[V], W))] = {
this.cogroup(other, partitioner).flatMapValues {
@@ -155,56 +224,117 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
}
- def combineByKey[C](createCombiner: V => C,
- mergeValue: (C, V) => C,
- mergeCombiners: (C, C) => C) : RDD[(K, C)] = {
+ /**
+ * Simplified version of combineByKey that hash-partitions the resulting RDD using the default
+ * parallelism level.
+ */
+ def combineByKey[C](createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C)
+ : RDD[(K, C)] = {
combineByKey(createCombiner, mergeValue, mergeCombiners, 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 the default parallelism level.
+ */
def reduceByKey(func: (V, V) => V): RDD[(K, V)] = {
reduceByKey(defaultPartitioner(self), func)
}
+ /**
+ * Group the values for each key in the RDD into a single sequence. Hash-partitions the
+ * resulting RDD with the default parallelism level.
+ */
def groupByKey(): RDD[(K, Seq[V])] = {
groupByKey(defaultPartitioner(self))
}
+ /**
+ * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
+ * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
+ * (k, v2) is in `other`. Performs a hash join across the cluster.
+ */
def join[W](other: RDD[(K, W)]): RDD[(K, (V, W))] = {
join(other, defaultPartitioner(self, other))
}
+ /**
+ * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
+ * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
+ * (k, v2) is in `other`. Performs a hash join across the cluster.
+ */
def join[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (V, W))] = {
join(other, new HashPartitioner(numSplits))
}
+ /**
+ * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
+ * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
+ * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
+ * using the default level of parallelism.
+ */
def leftOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (V, Option[W]))] = {
leftOuterJoin(other, defaultPartitioner(self, other))
}
+ /**
+ * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
+ * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
+ * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
+ * into `numSplits` partitions.
+ */
def leftOuterJoin[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (V, Option[W]))] = {
leftOuterJoin(other, new HashPartitioner(numSplits))
}
+ /**
+ * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
+ * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
+ * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
+ * RDD using the default parallelism level.
+ */
def rightOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], W))] = {
rightOuterJoin(other, defaultPartitioner(self, other))
}
+ /**
+ * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
+ * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
+ * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
+ * RDD into the given number of partitions.
+ */
def rightOuterJoin[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (Option[V], W))] = {
rightOuterJoin(other, new HashPartitioner(numSplits))
}
+ /**
+ * Return the key-value pairs in this RDD to the master as a Map.
+ */
def collectAsMap(): Map[K, V] = HashMap(self.collect(): _*)
-
+
+ /**
+ * Pass each value in the key-value pair RDD through a map function without changing the keys;
+ * this also retains the original RDD's partitioning.
+ */
def mapValues[U](f: V => U): RDD[(K, U)] = {
val cleanF = self.context.clean(f)
new MappedValuesRDD(self, cleanF)
}
-
+
+ /**
+ * Pass each value in the key-value pair RDD through a flatMap function without changing the
+ * keys; this also retains the original RDD's partitioning.
+ */
def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = {
val cleanF = self.context.clean(f)
new FlatMappedValuesRDD(self, cleanF)
}
-
+
+ /**
+ * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
+ * list of values for that key in `this` as well as `other`.
+ */
def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = {
val cg = new CoGroupedRDD[K](
Seq(self.asInstanceOf[RDD[(_, _)]], other.asInstanceOf[RDD[(_, _)]]),
@@ -215,12 +345,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
(vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]])
}
}
-
+
+ /**
+ * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
+ * tuple with the list of values for that key in `this`, `other1` and `other2`.
+ */
def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner)
: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = {
val cg = new CoGroupedRDD[K](
Seq(self.asInstanceOf[RDD[(_, _)]],
- other1.asInstanceOf[RDD[(_, _)]],
+ other1.asInstanceOf[RDD[(_, _)]],
other2.asInstanceOf[RDD[(_, _)]]),
partitioner)
val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
@@ -230,28 +364,46 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
}
+ /**
+ * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
+ * list of values for that key in `this` as well as `other`.
+ */
def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = {
cogroup(other, defaultPartitioner(self, other))
}
+ /**
+ * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
+ * tuple with the list of values for that key in `this`, `other1` and `other2`.
+ */
def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)])
: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = {
cogroup(other1, other2, defaultPartitioner(self, other1, other2))
}
+ /**
+ * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
+ * list of values for that key in `this` as well as `other`.
+ */
def cogroup[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (Seq[V], Seq[W]))] = {
cogroup(other, new HashPartitioner(numSplits))
}
+ /**
+ * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
+ * tuple with the list of values for that key in `this`, `other1` and `other2`.
+ */
def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numSplits: Int)
: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = {
cogroup(other1, other2, new HashPartitioner(numSplits))
}
+ /** Alias for cogroup. */
def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = {
cogroup(other, defaultPartitioner(self, other))
}
+ /** Alias for cogroup. */
def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)])
: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = {
cogroup(other1, other2, defaultPartitioner(self, other1, other2))
@@ -268,6 +420,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
return new HashPartitioner(self.context.defaultParallelism)
}
+ /**
+ * Return the list of values in the RDD for key `key`. This operation is done efficiently if the
+ * RDD has a known partitioner by only searching the partition that the key maps to.
+ */
def lookup(key: K): Seq[V] = {
self.partitioner match {
case Some(p) =>
@@ -286,14 +442,26 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
}
+ /**
+ * 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]])
}
-
+
+ /**
+ * 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]])
}
+ /**
+ * 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(
path: String,
keyClass: Class[_],
@@ -302,6 +470,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, new Configuration)
}
+ /**
+ * 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(
path: String,
keyClass: Class[_],
@@ -323,7 +495,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
/* "reduce task" <split #> <attempt # = spark task #> */
val attemptId = new TaskAttemptID(jobtrackerID,
stageId, false, context.splitId, attemptNumber)
- val hadoopContext = new TaskAttemptContext(wrappedConf.value, attemptId)
+ val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
val format = outputFormatClass.newInstance
val committer = format.getOutputCommitter(hadoopContext)
committer.setupTask(hadoopContext)
@@ -342,13 +514,17 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
* setupJob/commitJob, so we just use a dummy "map" task.
*/
val jobAttemptId = new TaskAttemptID(jobtrackerID, stageId, true, 0, 0)
- val jobTaskContext = new TaskAttemptContext(wrappedConf.value, jobAttemptId)
+ val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId)
val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext)
jobCommitter.setupJob(jobTaskContext)
val count = self.context.runJob(self, writeShard _).sum
jobCommitter.cleanupJob(jobTaskContext)
}
+ /**
+ * 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(
path: String,
keyClass: Class[_],
@@ -363,7 +539,13 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
FileOutputFormat.setOutputPath(conf, HadoopWriter.createPathFromString(path, conf))
saveAsHadoopDataset(conf)
}
-
+
+ /**
+ * Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for
+ * that storage system. The JobConf should set an OutputFormat and any output paths required
+ * (e.g. a table name to write to) in the same way as it would be configured for a Hadoop
+ * MapReduce job.
+ */
def saveAsHadoopDataset(conf: JobConf) {
val outputFormatClass = conf.getOutputFormat
val keyClass = conf.getOutputKeyClass
@@ -377,7 +559,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
if (valueClass == null) {
throw new SparkException("Output value class not set")
}
-
+
logInfo("Saving as hadoop file of type (" + keyClass.getSimpleName+ ", " + valueClass.getSimpleName+ ")")
val writer = new HadoopWriter(conf)
@@ -390,14 +572,14 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
writer.setup(context.stageId, context.splitId, attemptNumber)
writer.open()
-
+
var count = 0
while(iter.hasNext) {
val record = iter.next
count += 1
writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef])
}
-
+
writer.close()
writer.commit()
}
@@ -406,35 +588,42 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
writer.cleanup()
}
- def getKeyClass() = implicitly[ClassManifest[K]].erasure
+ private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure
- def getValueClass() = implicitly[ClassManifest[V]].erasure
+ private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure
}
+/**
+ * Extra functions available on RDDs of (key, value) pairs where the key is sortable through
+ * an implicit conversion. Import `spark.SparkContext._` at the top of your program to 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](
self: RDD[(K, V)])
- extends Logging
+ extends Logging
with Serializable {
- def sortByKey(ascending: Boolean = true): RDD[(K,V)] = {
- val rangePartitionedRDD = self.partitionBy(new RangePartitioner(self.splits.size, self, ascending))
- new SortedRDD(rangePartitionedRDD, ascending)
+ /**
+ * 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(ascending: Boolean = true, numSplits: Int = self.splits.size): RDD[(K,V)] = {
+ val shuffled =
+ new ShuffledRDD[K, V](self, new RangePartitioner(numSplits, self, ascending))
+ shuffled.mapPartitions(iter => {
+ val buf = iter.toArray
+ if (ascending) {
+ buf.sortWith((x, y) => x._1 < y._1).iterator
+ } else {
+ buf.sortWith((x, y) => x._1 > y._1).iterator
+ }
+ }, true)
}
}
-class SortedRDD[K <% Ordered[K], V](prev: RDD[(K, V)], ascending: Boolean)
- extends RDD[(K, V)](prev.context) {
-
- override def splits = prev.splits
- override val partitioner = prev.partitioner
- override val dependencies = List(new OneToOneDependency(prev))
-
- override def compute(split: Split) = {
- prev.iterator(split).toArray
- .sortWith((x, y) => if (ascending) x._1 < y._1 else x._1 > y._1).iterator
- }
-}
-
+private[spark]
class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev.context) {
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
@@ -442,9 +631,10 @@ class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)]
override def compute(split: Split) = prev.iterator(split).map{case (k, v) => (k, f(v))}
}
+private[spark]
class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U])
extends RDD[(K, U)](prev.context) {
-
+
override def splits = prev.splits
override val dependencies = List(new OneToOneDependency(prev))
override val partitioner = prev.partitioner
@@ -454,6 +644,6 @@ class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U]
}
}
-object Manifests {
+private[spark] object Manifests {
val seqSeqManifest = classManifest[Seq[Seq[_]]]
}
diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/ParallelCollection.scala
index d79007ab40..9b57ae3b4f 100644
--- a/core/src/main/scala/spark/ParallelCollection.scala
+++ b/core/src/main/scala/spark/ParallelCollection.scala
@@ -3,7 +3,7 @@ package spark
import scala.collection.immutable.NumericRange
import scala.collection.mutable.ArrayBuffer
-class ParallelCollectionSplit[T: ClassManifest](
+private[spark] class ParallelCollectionSplit[T: ClassManifest](
val rddId: Long,
val slice: Int,
values: Seq[T])
@@ -21,7 +21,7 @@ class ParallelCollectionSplit[T: ClassManifest](
override val index: Int = slice
}
-class ParallelCollection[T: ClassManifest](
+private[spark] class ParallelCollection[T: ClassManifest](
sc: SparkContext,
@transient data: Seq[T],
numSlices: Int)
diff --git a/core/src/main/scala/spark/Partitioner.scala b/core/src/main/scala/spark/Partitioner.scala
index 643541429f..b71021a082 100644
--- a/core/src/main/scala/spark/Partitioner.scala
+++ b/core/src/main/scala/spark/Partitioner.scala
@@ -1,10 +1,17 @@
package spark
+/**
+ * An object that defines how the elements in a key-value pair RDD are partitioned by key.
+ * Maps each key to a partition ID, from 0 to `numPartitions - 1`.
+ */
abstract class Partitioner extends Serializable {
def numPartitions: Int
def getPartition(key: Any): Int
}
+/**
+ * A [[spark.Partitioner]] that implements hash-based partitioning using Java's `Object.hashCode`.
+ */
class HashPartitioner(partitions: Int) extends Partitioner {
def numPartitions = partitions
@@ -29,6 +36,10 @@ class HashPartitioner(partitions: Int) extends Partitioner {
}
}
+/**
+ * A [[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](
partitions: Int,
@transient rdd: RDD[(K,V)],
@@ -41,9 +52,9 @@ class RangePartitioner[K <% Ordered[K]: ClassManifest, V](
Array()
} else {
val rddSize = rdd.count()
- val maxSampleSize = partitions * 10.0
+ val maxSampleSize = partitions * 20.0
val frac = math.min(maxSampleSize / math.max(rddSize, 1), 1.0)
- val rddSample = rdd.sample(true, frac, 1).map(_._1).collect().sortWith(_ < _)
+ val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sortWith(_ < _)
if (rddSample.length == 0) {
Array()
} else {
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index d28f3593fe..338dff4061 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -31,51 +31,86 @@ import spark.partial.BoundedDouble
import spark.partial.CountEvaluator
import spark.partial.GroupedCountEvaluator
import spark.partial.PartialResult
+import spark.rdd.BlockRDD
+import spark.rdd.CartesianRDD
+import spark.rdd.FilteredRDD
+import spark.rdd.FlatMappedRDD
+import spark.rdd.GlommedRDD
+import spark.rdd.MappedRDD
+import spark.rdd.MapPartitionsRDD
+import spark.rdd.MapPartitionsWithSplitRDD
+import spark.rdd.PipedRDD
+import spark.rdd.SampledRDD
+import spark.rdd.UnionRDD
import spark.storage.StorageLevel
import SparkContext._
/**
* A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable,
- * partitioned collection of elements that can be operated on in parallel.
+ * partitioned collection of elements that can be operated on in parallel. This class contains the
+ * basic operations available on all RDDs, such as `map`, `filter`, and `persist`. In addition,
+ * [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value pairs, such
+ * as `groupByKey` and `join`; [[spark.DoubleRDDFunctions]] contains operations available only on
+ * RDDs of Doubles; and [[spark.SequenceFileRDDFunctions]] contains operations available on RDDs
+ * that can be saved as SequenceFiles. These operations are automatically available on any RDD of
+ * the right type (e.g. RDD[(Int, Int)] through implicit conversions when you
+ * `import spark.SparkContext._`.
*
- * Each RDD is characterized by five main properties:
- * - A list of splits (partitions)
- * - A function for computing each split
- * - A list of dependencies on other RDDs
- * - Optionally, a Partitioner for key-value RDDs (e.g. to say that the RDD is hash-partitioned)
- * - Optionally, a list of preferred locations to compute each split on (e.g. block locations for
- * HDFS)
+ * Internally, each RDD is characterized by five main properties:
*
- * All the scheduling and execution in Spark is done based on these methods, allowing each RDD to
- * implement its own way of computing itself.
+ * - A list of splits (partitions)
+ * - A function for computing each split
+ * - A list of dependencies on other RDDs
+ * - Optionally, a Partitioner for key-value RDDs (e.g. to say that the RDD is hash-partitioned)
+ * - Optionally, a list of preferred locations to compute each split on (e.g. block locations for
+ * an HDFS file)
*
- * This class also contains transformation methods available on all RDDs (e.g. map and filter). In
- * addition, PairRDDFunctions contains extra methods available on RDDs of key-value pairs, and
- * SequenceFileRDDFunctions contains extra methods for saving RDDs to Hadoop SequenceFiles.
+ * All of the scheduling and execution in Spark is done based on these methods, allowing each RDD
+ * to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for
+ * reading data from a new storage system) by overriding these functions. Please refer to the
+ * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details
+ * on RDD internals.
*/
abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serializable {
- // Methods that must be implemented by subclasses
+ // Methods that must be implemented by subclasses:
+
+ /** Set of partitions in this RDD. */
def splits: Array[Split]
+
+ /** Function for computing a given partition. */
def compute(split: Split): Iterator[T]
+
+ /** How this RDD depends on any parent RDDs. */
@transient val dependencies: List[Dependency[_]]
+
+ // Methods available on all RDDs:
+
+ /** Record user function generating this RDD. */
+ private[spark] val origin = Utils.getSparkCallSite
- // Optionally overridden by subclasses to specify how they are partitioned
+ /** Optionally overridden by subclasses to specify how they are partitioned. */
val partitioner: Option[Partitioner] = None
- // Optionally overridden by subclasses to specify placement preferences
+ /** Optionally overridden by subclasses to specify placement preferences. */
def preferredLocations(split: Split): Seq[String] = Nil
+ /** The [[spark.SparkContext]] that this RDD was created on. */
def context = sc
+
+ private[spark] def elementClassManifest: ClassManifest[T] = classManifest[T]
- // Get a unique ID for this RDD
+ /** A unique ID for this RDD (within its SparkContext). */
val id = sc.newRddId()
// Variables relating to persistence
private var storageLevel: StorageLevel = StorageLevel.NONE
- // Change this RDD's storage level
+ /**
+ * 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.
+ */
def persist(newLevel: StorageLevel): RDD[T] = {
// TODO: Handle changes of StorageLevel
if (storageLevel != StorageLevel.NONE && newLevel != storageLevel) {
@@ -86,22 +121,23 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
this
}
- // Turn on the default caching level for this RDD
- def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY_DESER)
+ /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
+ def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY)
- // Turn on the default caching level for this RDD
+ /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
def cache(): RDD[T] = persist()
+ /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
def getStorageLevel = storageLevel
- def checkpoint(level: StorageLevel = StorageLevel.DISK_AND_MEMORY_DESER_2): RDD[T] = {
+ private[spark] def checkpoint(level: StorageLevel = StorageLevel.MEMORY_AND_DISK_2): RDD[T] = {
if (!level.useDisk && level.replication < 2) {
throw new Exception("Cannot checkpoint without using disk or replication (level requested was " + level + ")")
}
// This is a hack. Ideally this should re-use the code used by the CacheTracker
// to generate the key.
- def getSplitKey(split: Split) = "rdd:%d:%d".format(this.id, split.index)
+ def getSplitKey(split: Split) = "rdd_%d_%d".format(this.id, split.index)
persist(level)
sc.runJob(this, (iter: Iterator[T]) => {} )
@@ -113,7 +149,11 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
}
}
- // Read this RDD; will read from cache if applicable, or otherwise compute
+ /**
+ * Internal method to this RDD; will read from cache if applicable, or otherwise compute it.
+ * This should ''not'' be called by users directly, but is available for implementors of custom
+ * subclasses of RDD.
+ */
final def iterator(split: Split): Iterator[T] = {
if (storageLevel != StorageLevel.NONE) {
SparkEnv.get.cacheTracker.getOrCompute[T](this, split, storageLevel)
@@ -124,15 +164,32 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
// Transformations (return a new RDD)
+ /**
+ * 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))
-
+
+ /**
+ * 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] =
new FlatMappedRDD(this, sc.clean(f))
-
+
+ /**
+ * Return a new RDD containing only the elements that satisfy a predicate.
+ */
def filter(f: T => Boolean): RDD[T] = new FilteredRDD(this, sc.clean(f))
- def distinct(): RDD[T] = map(x => (x, "")).reduceByKey((x, y) => x).map(_._1)
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
+ def distinct(numSplits: Int = splits.size): RDD[T] =
+ map(x => (x, null)).reduceByKey((x, y) => x, numSplits).map(_._1)
+ /**
+ * Return a sampled subset of this RDD.
+ */
def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] =
new SampledRDD(this, withReplacement, fraction, seed)
@@ -143,8 +200,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
var initialCount = count()
var maxSelected = 0
- if (initialCount > Integer.MAX_VALUE) {
- maxSelected = Integer.MAX_VALUE
+ if (initialCount > Integer.MAX_VALUE - 1) {
+ maxSelected = Integer.MAX_VALUE - 1
} else {
maxSelected = initialCount.toInt
}
@@ -159,56 +216,109 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
total = num
}
- var samples = this.sample(withReplacement, fraction, seed).collect()
+ val rand = new Random(seed)
+ var samples = this.sample(withReplacement, fraction, rand.nextInt).collect()
while (samples.length < total) {
- samples = this.sample(withReplacement, fraction, seed).collect()
+ samples = this.sample(withReplacement, fraction, rand.nextInt).collect()
}
- val arr = samples.take(total)
-
- return arr
+ Utils.randomizeInPlace(samples, rand).take(total)
}
+ /**
+ * Return the union of this RDD and another one. Any identical elements will appear multiple
+ * times (use `.distinct()` to eliminate them).
+ */
def union(other: RDD[T]): RDD[T] = new UnionRDD(sc, Array(this, other))
+ /**
+ * Return the union of this RDD and another one. Any identical elements will appear multiple
+ * times (use `.distinct()` to eliminate them).
+ */
def ++(other: RDD[T]): RDD[T] = this.union(other)
+ /**
+ * Return an RDD created by coalescing all elements within each partition into an array.
+ */
def glom(): RDD[Array[T]] = new GlommedRDD(this)
+ /**
+ * 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)
+ /**
+ * 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, numSplits: Int): RDD[(K, Seq[T])] = {
val cleanF = sc.clean(f)
this.map(t => (cleanF(t), t)).groupByKey(numSplits)
}
+ /**
+ * Return an RDD of grouped items.
+ */
def groupBy[K: ClassManifest](f: T => K): RDD[(K, Seq[T])] = groupBy[K](f, sc.defaultParallelism)
+ /**
+ * Return an RDD created by piping elements to a forked external process.
+ */
def pipe(command: String): RDD[String] = new PipedRDD(this, command)
+ /**
+ * Return an RDD created by piping elements to a forked external process.
+ */
def pipe(command: Seq[String]): RDD[String] = new PipedRDD(this, command)
+ /**
+ * Return an RDD created by piping elements to a forked external process.
+ */
def pipe(command: Seq[String], env: Map[String, String]): RDD[String] =
new PipedRDD(this, command, env)
- def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U]): RDD[U] =
- new MapPartitionsRDD(this, sc.clean(f))
+ /**
+ * Return a new RDD by applying a function to each partition of this RDD.
+ */
+ def mapPartitions[U: ClassManifest](f: Iterator[T] => Iterator[U],
+ preservesPartitioning: Boolean = false): RDD[U] =
+ new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning)
+
+ /**
+ * Return a new RDD by applying a function to each partition of this RDD, while tracking the index
+ * of the original partition.
+ */
+ def mapPartitionsWithSplit[U: ClassManifest](f: (Int, Iterator[T]) => Iterator[U]): RDD[U] =
+ new MapPartitionsWithSplitRDD(this, sc.clean(f))
// Actions (launch a job to return a value to the user program)
-
+
+ /**
+ * Applies a function f to all elements of this RDD.
+ */
def foreach(f: T => Unit) {
val cleanF = sc.clean(f)
sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF))
}
+ /**
+ * Return an array that contains all of the elements in this RDD.
+ */
def collect(): Array[T] = {
val results = sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
Array.concat(results: _*)
}
+ /**
+ * Return an array that contains all of the elements in this RDD.
+ */
def toArray(): Array[T] = collect()
+ /**
+ * Reduces the elements of this RDD using the specified associative binary operator.
+ */
def reduce(f: (T, T) => T): T = {
val cleanF = sc.clean(f)
val reducePartition: Iterator[T] => Option[T] = iter => {
@@ -257,7 +367,10 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
(iter: Iterator[T]) => iter.aggregate(zeroValue)(cleanSeqOp, cleanCombOp))
return results.fold(zeroValue)(cleanCombOp)
}
-
+
+ /**
+ * Return the number of elements in the RDD.
+ */
def count(): Long = {
sc.runJob(this, (iter: Iterator[T]) => {
var result = 0L
@@ -270,7 +383,8 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
}
/**
- * Approximate version of count() that returns a potentially incomplete result after a timeout.
+ * (Experimental) Approximate version of count() that returns a potentially incomplete result
+ * within a timeout, even if not all tasks have finished.
*/
def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) =>
@@ -286,12 +400,11 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
}
/**
- * Count elements equal to each value, returning a map of (value, count) pairs. The final combine
- * step happens locally on the master, equivalent to running a single reduce task.
- *
- * TODO: This should perhaps be distributed by default.
+ * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final
+ * combine step happens locally on the master, equivalent to running a single reduce task.
*/
def countByValue(): Map[T, Long] = {
+ // TODO: This should perhaps be distributed by default.
def countPartition(iter: Iterator[T]): Iterator[OLMap[T]] = {
val map = new OLMap[T]
while (iter.hasNext) {
@@ -313,7 +426,7 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
}
/**
- * Approximate version of countByValue().
+ * (Experimental) Approximate version of countByValue().
*/
def countByValueApprox(
timeout: Long,
@@ -353,18 +466,27 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
return buf.toArray
}
+ /**
+ * Return the first element in this RDD.
+ */
def first(): T = take(1) match {
case Array(t) => t
case _ => throw new UnsupportedOperationException("empty collection")
}
+ /**
+ * Save this RDD as a text file, using string representations of elements.
+ */
def saveAsTextFile(path: String) {
this.map(x => (NullWritable.get(), new Text(x.toString)))
.saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path)
}
+ /**
+ * Save this RDD as a SequenceFile of serialized objects.
+ */
def saveAsObjectFile(path: String) {
- this.glom
+ this.mapPartitions(iter => iter.grouped(10).map(_.toArray))
.map(x => (NullWritable.get(), new BytesWritable(Utils.serialize(x))))
.saveAsSequenceFile(path)
}
@@ -374,45 +496,3 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial
sc.runJob(this, (iter: Iterator[T]) => iter.toArray)
}
}
-
-class MappedRDD[U: ClassManifest, T: ClassManifest](
- prev: RDD[T],
- f: T => U)
- extends RDD[U](prev.context) {
-
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override def compute(split: Split) = prev.iterator(split).map(f)
-}
-
-class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
- prev: RDD[T],
- f: T => TraversableOnce[U])
- extends RDD[U](prev.context) {
-
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override def compute(split: Split) = prev.iterator(split).flatMap(f)
-}
-
-class FilteredRDD[T: ClassManifest](prev: RDD[T], f: T => Boolean) extends RDD[T](prev.context) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override def compute(split: Split) = prev.iterator(split).filter(f)
-}
-
-class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev.context) {
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override def compute(split: Split) = Array(prev.iterator(split).toArray).iterator
-}
-
-class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
- prev: RDD[T],
- f: Iterator[T] => Iterator[U])
- extends RDD[U](prev.context) {
-
- override def splits = prev.splits
- override val dependencies = List(new OneToOneDependency(prev))
- override def compute(split: Split) = f(prev.iterator(split))
-}
diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
index ea7171d3a1..a34aee69c1 100644
--- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
+++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala
@@ -23,19 +23,21 @@ import org.apache.hadoop.io.NullWritable
import org.apache.hadoop.io.BytesWritable
import org.apache.hadoop.io.Text
-import SparkContext._
+import spark.SparkContext._
/**
* Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile,
* through an implicit conversion. Note that this can't be part of PairRDDFunctions because
* we need more implicit parameters to convert our keys and values to Writable.
+ *
+ * Users should import `spark.SparkContext._` at the top of their program to use these functions.
*/
class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : ClassManifest](
self: RDD[(K, V)])
extends Logging
with Serializable {
- def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = {
+ private def getWritableClass[T <% Writable: ClassManifest](): Class[_ <: Writable] = {
val c = {
if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) {
classManifest[T].erasure
@@ -47,6 +49,13 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla
c.asInstanceOf[Class[_ <: Writable]]
}
+ /**
+ * Output the RDD as a Hadoop SequenceFile using the Writable types we infer from the RDD's key
+ * and value types. If the key or value are Writable, then we use their classes directly;
+ * otherwise we map primitive types such as Int and Double to IntWritable, DoubleWritable, etc,
+ * byte arrays to BytesWritable, and Strings to Text. The `path` can be on any Hadoop-supported
+ * file system.
+ */
def saveAsSequenceFile(path: String) {
def anyToWritable[U <% Writable](u: U): Writable = u
diff --git a/core/src/main/scala/spark/ShuffleFetcher.scala b/core/src/main/scala/spark/ShuffleFetcher.scala
index 4f8d98f7d0..d9a94d4021 100644
--- a/core/src/main/scala/spark/ShuffleFetcher.scala
+++ b/core/src/main/scala/spark/ShuffleFetcher.scala
@@ -1,10 +1,12 @@
package spark
-abstract class ShuffleFetcher {
- // Fetch the shuffle outputs for a given ShuffleDependency, calling func exactly
- // once on each key-value pair obtained.
- def fetch[K, V](shuffleId: Int, reduceId: Int, func: (K, V) => Unit)
+private[spark] abstract class ShuffleFetcher {
+ /**
+ * Fetch the shuffle outputs for a given ShuffleDependency.
+ * @return An iterator over the elements of the fetched shuffle outputs.
+ */
+ def fetch[K, V](shuffleId: Int, reduceId: Int) : Iterator[(K, V)]
- // Stop the fetcher
+ /** Stop the fetcher */
def stop() {}
}
diff --git a/core/src/main/scala/spark/ShuffleManager.scala b/core/src/main/scala/spark/ShuffleManager.scala
deleted file mode 100644
index 24af7f3a08..0000000000
--- a/core/src/main/scala/spark/ShuffleManager.scala
+++ /dev/null
@@ -1,98 +0,0 @@
-package spark
-
-import java.io._
-import java.net.URL
-import java.util.UUID
-import java.util.concurrent.atomic.AtomicLong
-
-import scala.collection.mutable.{ArrayBuffer, HashMap}
-
-import spark._
-
-class ShuffleManager extends Logging {
- private var nextShuffleId = new AtomicLong(0)
-
- private var shuffleDir: File = null
- private var server: HttpServer = null
- private var serverUri: String = null
-
- initialize()
-
- private def initialize() {
- // TODO: localDir should be created by some mechanism common to Spark
- // so that it can be shared among shuffle, broadcast, etc
- val localDirRoot = System.getProperty("spark.local.dir", "/tmp")
- var tries = 0
- var foundLocalDir = false
- var localDir: File = null
- var localDirUuid: UUID = null
- while (!foundLocalDir && tries < 10) {
- tries += 1
- try {
- localDirUuid = UUID.randomUUID
- localDir = new File(localDirRoot, "spark-local-" + localDirUuid)
- if (!localDir.exists) {
- localDir.mkdirs()
- foundLocalDir = true
- }
- } catch {
- case e: Exception =>
- logWarning("Attempt " + tries + " to create local dir failed", e)
- }
- }
- if (!foundLocalDir) {
- logError("Failed 10 attempts to create local dir in " + localDirRoot)
- System.exit(1)
- }
- shuffleDir = new File(localDir, "shuffle")
- shuffleDir.mkdirs()
- logInfo("Shuffle dir: " + shuffleDir)
-
- // Add a shutdown hook to delete the local dir
- Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dir") {
- override def run() {
- Utils.deleteRecursively(localDir)
- }
- })
-
- val extServerPort = System.getProperty(
- "spark.localFileShuffle.external.server.port", "-1").toInt
- if (extServerPort != -1) {
- // We're using an external HTTP server; set URI relative to its root
- var extServerPath = System.getProperty(
- "spark.localFileShuffle.external.server.path", "")
- if (extServerPath != "" && !extServerPath.endsWith("/")) {
- extServerPath += "/"
- }
- serverUri = "http://%s:%d/%s/spark-local-%s".format(
- Utils.localIpAddress, extServerPort, extServerPath, localDirUuid)
- } else {
- // Create our own server
- server = new HttpServer(localDir)
- server.start()
- serverUri = server.uri
- }
- logInfo("Local URI: " + serverUri)
- }
-
- def stop() {
- if (server != null) {
- server.stop()
- }
- }
-
- def getOutputFile(shuffleId: Long, inputId: Int, outputId: Int): File = {
- val dir = new File(shuffleDir, shuffleId + "/" + inputId)
- dir.mkdirs()
- val file = new File(dir, "" + outputId)
- return file
- }
-
- def getServerUri(): String = {
- serverUri
- }
-
- def newShuffleId(): Long = {
- nextShuffleId.getAndIncrement()
- }
-}
diff --git a/core/src/main/scala/spark/ShuffledRDD.scala b/core/src/main/scala/spark/ShuffledRDD.scala
deleted file mode 100644
index 594dbd235f..0000000000
--- a/core/src/main/scala/spark/ShuffledRDD.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-package spark
-
-import java.util.{HashMap => JHashMap}
-
-class ShuffledRDDSplit(val idx: Int) extends Split {
- override val index = idx
- override def hashCode(): Int = idx
-}
-
-class ShuffledRDD[K, V, C](
- @transient parent: RDD[(K, V)],
- aggregator: Aggregator[K, V, C],
- part : Partitioner)
- extends RDD[(K, C)](parent.context) {
- //override val partitioner = Some(part)
- override val partitioner = Some(part)
-
- @transient
- val splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
-
- override def splits = splits_
-
- override def preferredLocations(split: Split) = Nil
-
- val dep = new ShuffleDependency(context.newShuffleId, parent, aggregator, part)
- override val dependencies = List(dep)
-
- override def compute(split: Split): Iterator[(K, C)] = {
- val combiners = new JHashMap[K, C]
- def mergePair(k: K, c: C) {
- val oldC = combiners.get(k)
- if (oldC == null) {
- combiners.put(k, c)
- } else {
- combiners.put(k, aggregator.mergeCombiners(oldC, c))
- }
- }
- val fetcher = SparkEnv.get.shuffleFetcher
- fetcher.fetch[K, C](dep.shuffleId, split.index, mergePair)
- return new Iterator[(K, C)] {
- var iter = combiners.entrySet().iterator()
-
- def hasNext: Boolean = iter.hasNext()
-
- def next(): (K, C) = {
- val entry = iter.next()
- (entry.getKey, entry.getValue)
- }
- }
- }
-}
diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala
index e5ad8b52dc..7c3e8640e9 100644
--- a/core/src/main/scala/spark/SizeEstimator.scala
+++ b/core/src/main/scala/spark/SizeEstimator.scala
@@ -22,7 +22,7 @@ import it.unimi.dsi.fastutil.ints.IntOpenHashSet
* Based on the following JavaWorld article:
* http://www.javaworld.com/javaworld/javaqa/2003-12/02-qa-1226-sizeof.html
*/
-object SizeEstimator extends Logging {
+private[spark] object SizeEstimator extends Logging {
// Sizes of primitive types
private val BYTE_SIZE = 1
@@ -77,22 +77,18 @@ object SizeEstimator extends Logging {
return System.getProperty("spark.test.useCompressedOops").toBoolean
}
try {
- val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic";
- val server = ManagementFactory.getPlatformMBeanServer();
+ val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic"
+ val server = ManagementFactory.getPlatformMBeanServer()
val bean = ManagementFactory.newPlatformMXBeanProxy(server,
- hotSpotMBeanName, classOf[HotSpotDiagnosticMXBean]);
+ hotSpotMBeanName, classOf[HotSpotDiagnosticMXBean])
return bean.getVMOption("UseCompressedOops").getValue.toBoolean
} catch {
- case e: IllegalArgumentException => {
- logWarning("Exception while trying to check if compressed oops is enabled", e)
- // Fall back to checking if maxMemory < 32GB
- return Runtime.getRuntime.maxMemory < (32L*1024*1024*1024)
- }
-
- case e: SecurityException => {
- logWarning("No permission to create MBeanServer", e)
- // Fall back to checking if maxMemory < 32GB
- return Runtime.getRuntime.maxMemory < (32L*1024*1024*1024)
+ case e: Exception => {
+ // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB
+ val guess = Runtime.getRuntime.maxMemory < (32L*1024*1024*1024)
+ val guessInWords = if (guess) "yes" else "not"
+ logWarning("Failed to check whether UseCompressedOops is set; assuming " + guessInWords)
+ return guess
}
}
}
@@ -146,6 +142,10 @@ object SizeEstimator extends Logging {
val cls = obj.getClass
if (cls.isArray) {
visitArray(obj, cls, state)
+ } else if (obj.isInstanceOf[ClassLoader] || obj.isInstanceOf[Class[_]]) {
+ // Hadoop JobConfs created in the interpreter have a ClassLoader, which greatly confuses
+ // the size estimator since it references the whole REPL. Do nothing in this case. In
+ // general all ClassLoaders and Classes will be shared between objects anyway.
} else {
val classInfo = getClassInfo(cls)
state.size += classInfo.shellSize
diff --git a/core/src/main/scala/spark/SoftReferenceCache.scala b/core/src/main/scala/spark/SoftReferenceCache.scala
index ce9370c5d7..3dd0a4b1f9 100644
--- a/core/src/main/scala/spark/SoftReferenceCache.scala
+++ b/core/src/main/scala/spark/SoftReferenceCache.scala
@@ -5,7 +5,7 @@ import com.google.common.collect.MapMaker
/**
* An implementation of Cache that uses soft references.
*/
-class SoftReferenceCache extends Cache {
+private[spark] class SoftReferenceCache extends Cache {
val map = new MapMaker().softValues().makeMap[Any, Any]()
override def get(datasetId: Any, partition: Int): Any =
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index 1d5131ad13..0d37075ef3 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -2,13 +2,15 @@ package spark
import java.io._
import java.util.concurrent.atomic.AtomicInteger
+import java.net.{URI, URLClassLoader}
+
+import scala.collection.Map
+import scala.collection.generic.Growable
+import scala.collection.mutable.{ArrayBuffer, HashMap}
import akka.actor.Actor
import akka.actor.Actor._
-
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.{FileUtil, Path}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.InputFormat
import org.apache.hadoop.mapred.SequenceFileInputFormat
@@ -25,34 +27,59 @@ import org.apache.hadoop.io.Text
import org.apache.hadoop.mapred.FileInputFormat
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapred.TextInputFormat
-
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
import org.apache.hadoop.mapreduce.{Job => NewHadoopJob}
-
import org.apache.mesos.{Scheduler, MesosNativeLibrary}
import spark.broadcast._
-
+import spark.deploy.LocalSparkCluster
import spark.partial.ApproximateEvaluator
import spark.partial.PartialResult
-
+import spark.rdd.HadoopRDD
+import spark.rdd.NewHadoopRDD
+import spark.rdd.UnionRDD
import spark.scheduler.ShuffleMapTask
import spark.scheduler.DAGScheduler
import spark.scheduler.TaskScheduler
import spark.scheduler.local.LocalScheduler
import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler}
import spark.scheduler.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
-import spark.storage.BlockManagerMaster
+/**
+ * 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 jobName A name for your job, 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.
+ */
class SparkContext(
val master: String,
- val frameworkName: String,
+ val jobName: String,
val sparkHome: String,
- val jars: Seq[String])
+ val jars: Seq[String],
+ environment: Map[String, String])
extends Logging {
-
- def this(master: String, frameworkName: String) = this(master, frameworkName, null, Nil)
+
+ /**
+ * @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 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.
+ */
+ def this(master: String, jobName: String, sparkHome: String, jars: Seq[String]) =
+ this(master, jobName, sparkHome, jars, Map())
+
+ /**
+ * @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
+ */
+ def this(master: String, jobName: String) = this(master, jobName, null, Nil, Map())
// Ensure logging is initialized before we spawn any threads
initLogging()
@@ -68,46 +95,89 @@ class SparkContext(
private val isLocal = (master == "local" || master.startsWith("local["))
// Create the Spark execution environment (cache, map output tracker, etc)
- val env = SparkEnv.createFromSystemProperties(
+ private[spark] val env = SparkEnv.createFromSystemProperties(
System.getProperty("spark.master.host"),
System.getProperty("spark.master.port").toInt,
true,
isLocal)
SparkEnv.set(env)
+ // Used to store a URL for each static file/jar together with the file's local timestamp
+ private[spark] val addedFiles = HashMap[String, Long]()
+ private[spark] val addedJars = HashMap[String, Long]()
+
+ // Add each JAR given through the constructor
+ jars.foreach { addJar(_) }
+
+ // Environment variables to pass to our executors
+ private[spark] val executorEnvs = HashMap[String, String]()
+ for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS",
+ "SPARK_TESTING")) {
+ val value = System.getenv(key)
+ if (value != null) {
+ executorEnvs(key) = value
+ }
+ }
+ executorEnvs ++= environment
+
// Create and start the scheduler
private 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]+),([0-9]+)\]""".r
+ 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
master match {
- case "local" =>
- new LocalScheduler(1, 0)
+ case "local" =>
+ new LocalScheduler(1, 0, this)
- case LOCAL_N_REGEX(threads) =>
- new LocalScheduler(threads.toInt, 0)
+ case LOCAL_N_REGEX(threads) =>
+ new LocalScheduler(threads.toInt, 0, this)
case LOCAL_N_FAILURES_REGEX(threads, maxFailures) =>
- new LocalScheduler(threads.toInt, maxFailures.toInt)
+ new LocalScheduler(threads.toInt, maxFailures.toInt, this)
case SPARK_REGEX(sparkUrl) =>
val scheduler = new ClusterScheduler(this)
- val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, frameworkName)
+ val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName)
scheduler.initialize(backend)
scheduler
+ case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) =>
+ // Check to make sure SPARK_MEM <= memoryPerSlave. Otherwise Spark will just hang.
+ val memoryPerSlaveInt = memoryPerSlave.toInt
+ val sparkMemEnv = System.getenv("SPARK_MEM")
+ val sparkMemEnvInt = if (sparkMemEnv != null) Utils.memoryStringToMb(sparkMemEnv) else 512
+ if (sparkMemEnvInt > memoryPerSlaveInt) {
+ throw new SparkException(
+ "Slave memory (%d MB) cannot be smaller than SPARK_MEM (%d MB)".format(
+ memoryPerSlaveInt, sparkMemEnvInt))
+ }
+
+ val scheduler = new ClusterScheduler(this)
+ val localCluster = new LocalSparkCluster(
+ numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
+ val sparkUrl = localCluster.start()
+ val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName)
+ scheduler.initialize(backend)
+ backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
+ localCluster.stop()
+ }
+ scheduler
+
case _ =>
MesosNativeLibrary.load()
val scheduler = new ClusterScheduler(this)
val coarseGrained = System.getProperty("spark.mesos.coarse", "false").toBoolean
+ val masterWithoutProtocol = master.replaceFirst("^mesos://", "") // Strip initial mesos://
val backend = if (coarseGrained) {
- new CoarseMesosSchedulerBackend(scheduler, this, master, frameworkName)
+ new CoarseMesosSchedulerBackend(scheduler, this, masterWithoutProtocol, jobName)
} else {
- new MesosSchedulerBackend(scheduler, this, master, frameworkName)
+ new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, jobName)
}
scheduler.initialize(backend)
scheduler
@@ -119,14 +189,20 @@ class SparkContext(
// Methods for creating RDDs
- def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism ): RDD[T] = {
+ /** Distribute a local Scala collection to form an RDD. */
+ def parallelize[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
new ParallelCollection[T](this, seq, numSlices)
}
-
- def makeRDD[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism ): RDD[T] = {
+
+ /** Distribute a local Scala collection to form an RDD. */
+ def makeRDD[T: ClassManifest](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = {
parallelize(seq, numSlices)
}
+ /**
+ * Read a text file from HDFS, a local file system (available on all nodes), or any
+ * Hadoop-supported file system URI, and return it as an RDD of Strings.
+ */
def textFile(path: String, minSplits: Int = defaultMinSplits): RDD[String] = {
hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text], minSplits)
.map(pair => pair._2.toString)
@@ -163,19 +239,31 @@ class SparkContext(
}
/**
- * Smarter version of hadoopFile() that uses class manifests to figure out the classes of keys,
- * values and the InputFormat so that users don't need to pass them directly.
+ * Smarter version of hadoopFile() that uses class manifests 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,
+ * {{{
+ * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minSplits)
+ * }}}
*/
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String, minSplits: Int)
(implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F])
: RDD[(K, V)] = {
hadoopFile(path,
- fm.erasure.asInstanceOf[Class[F]],
+ fm.erasure.asInstanceOf[Class[F]],
km.erasure.asInstanceOf[Class[K]],
vm.erasure.asInstanceOf[Class[V]],
minSplits)
}
+ /**
+ * Smarter version of hadoopFile() that uses class manifests 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,
+ * {{{
+ * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path)
+ * }}}
+ */
def hadoopFile[K, V, F <: InputFormat[K, V]](path: String)
(implicit km: ClassManifest[K], vm: ClassManifest[V], fm: ClassManifest[F]): RDD[(K, V)] =
hadoopFile[K, V, F](path, defaultMinSplits)
@@ -191,7 +279,7 @@ class SparkContext(
new Configuration)
}
- /**
+ /**
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
*/
@@ -207,7 +295,7 @@ class SparkContext(
new NewHadoopRDD(this, fClass, kClass, vClass, updatedConf)
}
- /**
+ /**
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
*/
@@ -219,7 +307,7 @@ class SparkContext(
new NewHadoopRDD(this, fClass, kClass, vClass, conf)
}
- /** Get an RDD for a Hadoop SequenceFile with given key and value types */
+ /** Get an RDD for a Hadoop SequenceFile with given key and value types. */
def sequenceFile[K, V](path: String,
keyClass: Class[K],
valueClass: Class[V],
@@ -229,18 +317,23 @@ class SparkContext(
hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits)
}
+ /** Get an RDD for a Hadoop SequenceFile with given key and value types. */
def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]): RDD[(K, V)] =
sequenceFile(path, keyClass, valueClass, defaultMinSplits)
/**
- * Version of sequenceFile() for types implicitly convertible to Writables through a
- * WritableConverter.
+ * Version of sequenceFile() for types implicitly convertible to Writables through a
+ * WritableConverter. For example, to access a SequenceFile where the keys are Text and the
+ * values are IntWritable, you could simply write
+ * {{{
+ * sparkContext.sequenceFile[String, Int](path, ...)
+ * }}}
*
* WritableConverters are provided in a somewhat strange way (by an implicit function) to support
- * both subclasses of Writable and types for which we define a converter (e.g. Int to
+ * both subclasses of Writable and types for which we define a converter (e.g. Int to
* 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
+ * 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
* allow it to figure out the Writable class to use in the subclass case.
*/
@@ -265,7 +358,7 @@ class SparkContext(
* that there's very little effort required to save arbitrary objects.
*/
def objectFile[T: ClassManifest](
- path: String,
+ path: String,
minSplits: Int = defaultMinSplits
): RDD[T] = {
sequenceFile(path, classOf[NullWritable], classOf[BytesWritable], minSplits)
@@ -275,43 +368,128 @@ class SparkContext(
/** Build the union of a list of RDDs. */
def union[T: ClassManifest](rdds: Seq[RDD[T]]): RDD[T] = new UnionRDD(this, rdds)
- /** Build the union of a list of 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] =
new UnionRDD(this, Seq(first) ++ rest)
// Methods for creating shared variables
+ /**
+ * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values
+ * to using the `+=` method. Only the master can access the accumulator's `value`.
+ */
def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) =
new Accumulator(initialValue, param)
/**
- * Create an accumulable shared variable, with a `+=` method
+ * Create an [[spark.Accumulable]] shared variable, with a `+=` method
* @tparam T accumulator type
* @tparam R type that can be added to the accumulator
*/
def accumulable[T,R](initialValue: T)(implicit param: AccumulableParam[T,R]) =
new Accumulable(initialValue, param)
+ /**
+ * Create an accumulator from a "mutable collection" type.
+ *
+ * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by
+ * standard mutable collections. So you can use this with mutable Map, Set, etc.
+ */
+ def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T](initialValue: R) = {
+ val param = new GrowableAccumulableParam[R,T]
+ new Accumulable(initialValue, param)
+ }
+
+ /**
+ * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for
+ * reading it in distributed functions. The variable will be sent to each cluster only once.
+ */
+ def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T] (value, isLocal)
+
+ /**
+ * Add a file to be downloaded into the working directory of this Spark job on every node.
+ * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
+ * filesystems), or an HTTP, HTTPS or FTP URI.
+ */
+ def addFile(path: String) {
+ val uri = new URI(path)
+ val key = uri.getScheme match {
+ case null | "file" => env.httpFileServer.addFile(new File(uri.getPath))
+ case _ => path
+ }
+ addedFiles(key) = System.currentTimeMillis
- // Keep around a weak hash map of values to Cached versions?
- def broadcast[T](value: T) = SparkEnv.get.broadcastManager.newBroadcast[T] (value, isLocal)
+ // Fetch the file locally in case the task is executed locally
+ val filename = new File(path.split("/").last)
+ Utils.fetchFile(path, new File("."))
- // Stop the SparkContext
+ logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
+ }
+
+ /**
+ * Return a map from the slave to the max memory available for caching and the remaining
+ * memory available for caching.
+ */
+ def getSlavesMemoryStatus: Map[String, (Long, Long)] = {
+ env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) =>
+ (blockManagerId.ip + ":" + blockManagerId.port, mem)
+ }
+ }
+
+ /**
+ * Clear the job's list of files added by `addFile` so that they do not get donwloaded to
+ * any new nodes.
+ */
+ def clearFiles() {
+ addedFiles.keySet.map(_.split("/").last).foreach { k => new File(k).delete() }
+ addedFiles.clear()
+ }
+
+ /**
+ * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.
+ * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
+ * filesystems), or an HTTP, HTTPS or FTP URI.
+ */
+ def addJar(path: String) {
+ val uri = new URI(path)
+ val key = uri.getScheme match {
+ case null | "file" => env.httpFileServer.addJar(new File(uri.getPath))
+ case _ => path
+ }
+ addedJars(key) = System.currentTimeMillis
+ logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key))
+ }
+
+ /**
+ * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to
+ * any new nodes.
+ */
+ def clearJars() {
+ addedJars.keySet.map(_.split("/").last).foreach { k => new File(k).delete() }
+ addedJars.clear()
+ }
+
+ /** Shut down the SparkContext. */
def stop() {
dagScheduler.stop()
dagScheduler = null
taskScheduler = null
// TODO: Cache.stop()?
env.stop()
+ // Clean up locally linked files
+ clearFiles()
+ clearJars()
SparkEnv.set(null)
ShuffleMapTask.clearCache()
logInfo("Successfully stopped SparkContext")
}
- // Get Spark's home location from either a value set through the constructor,
- // or the spark.home Java property, or the SPARK_HOME environment variable
- // (in that order of preference). If neither of these is set, return None.
- def getSparkHome(): Option[String] = {
+ /**
+ * Get Spark's home location from either a value set through the constructor,
+ * or the spark.home Java property, or the SPARK_HOME environment variable
+ * (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) {
@@ -326,7 +504,7 @@ class SparkContext(
/**
* Run a function on a given set of partitions in an RDD and return the results. This is the main
* entry point to the scheduler, by which all actions get launched. The allowLocal flag specifies
- * whether the scheduler can run the computation on the master rather than shipping it out to the
+ * whether the scheduler can run the computation on the master rather than shipping it out to the
* cluster, for short actions like first().
*/
def runJob[T, U: ClassManifest](
@@ -335,22 +513,27 @@ class SparkContext(
partitions: Seq[Int],
allowLocal: Boolean
): Array[U] = {
- logInfo("Starting job...")
+ val callSite = Utils.getSparkCallSite
+ logInfo("Starting job: " + callSite)
val start = System.nanoTime
- val result = dagScheduler.runJob(rdd, func, partitions, allowLocal)
- logInfo("Job finished in " + (System.nanoTime - start) / 1e9 + " s")
+ val result = dagScheduler.runJob(rdd, func, partitions, callSite, allowLocal)
+ logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
result
}
+ /**
+ * 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](
rdd: RDD[T],
- func: Iterator[T] => U,
+ func: Iterator[T] => U,
partitions: Seq[Int],
allowLocal: Boolean
): Array[U] = {
runJob(rdd, (context: TaskContext, iter: Iterator[T]) => func(iter), partitions, allowLocal)
}
-
+
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
@@ -358,6 +541,9 @@ class SparkContext(
runJob(rdd, func, 0 until rdd.splits.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] = {
runJob(rdd, func, 0 until rdd.splits.size, false)
}
@@ -371,38 +557,37 @@ class SparkContext(
evaluator: ApproximateEvaluator[U, R],
timeout: Long
): PartialResult[R] = {
- logInfo("Starting job...")
+ val callSite = Utils.getSparkCallSite
+ logInfo("Starting job: " + callSite)
val start = System.nanoTime
- val result = dagScheduler.runApproximateJob(rdd, func, evaluator, timeout)
- logInfo("Job finished in " + (System.nanoTime - start) / 1e9 + " s")
+ val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout)
+ logInfo("Job finished: " + callSite + ", took " + (System.nanoTime - start) / 1e9 + " s")
result
}
- // Clean a closure to make it ready to serialized and send to tasks
- // (removes unreferenced variables in $outer's, updates REPL variables)
+ /**
+ * Clean a closure to make it ready to serialized and send to tasks
+ * (removes unreferenced variables in $outer's, updates REPL variables)
+ */
private[spark] def clean[F <: AnyRef](f: F): F = {
ClosureCleaner.clean(f)
return f
}
- // Default level of parallelism to use when not given by user (e.g. for reduce tasks)
+ /** Default level of parallelism to use when not given by user (e.g. for reduce tasks) */
def defaultParallelism: Int = taskScheduler.defaultParallelism
- // Default min number of splits for Hadoop RDDs when not given by user
+ /** Default min number of splits for Hadoop RDDs when not given by user */
def defaultMinSplits: Int = math.min(defaultParallelism, 2)
private var nextShuffleId = new AtomicInteger(0)
- private[spark] def newShuffleId(): Int = {
- nextShuffleId.getAndIncrement()
- }
-
+ private[spark] def newShuffleId(): Int = nextShuffleId.getAndIncrement()
+
private var nextRddId = new AtomicInteger(0)
- // Register a new RDD, returning its RDD ID
- private[spark] def newRddId(): Int = {
- nextRddId.getAndIncrement()
- }
+ /** Register a new RDD, returning its RDD ID */
+ private[spark] def newRddId(): Int = nextRddId.getAndIncrement()
}
/**
@@ -429,7 +614,7 @@ object SparkContext {
implicit def rddToPairRDDFunctions[K: ClassManifest, V: ClassManifest](rdd: RDD[(K, V)]) =
new PairRDDFunctions(rdd)
-
+
implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable: ClassManifest](
rdd: RDD[(K, V)]) =
new SequenceFileRDDFunctions(rdd)
@@ -450,7 +635,7 @@ object SparkContext {
implicit def longToLongWritable(l: Long) = new LongWritable(l)
implicit def floatToFloatWritable(f: Float) = new FloatWritable(f)
-
+
implicit def doubleToDoubleWritable(d: Double) = new DoubleWritable(d)
implicit def boolToBoolWritable (b: Boolean) = new BooleanWritable(b)
@@ -461,7 +646,7 @@ object SparkContext {
private implicit def arrayToArrayWritable[T <% Writable: ClassManifest](arr: Traversable[T]): ArrayWritable = {
def anyToWritable[U <% Writable](u: U): Writable = u
-
+
new ArrayWritable(classManifest[T].erasure.asInstanceOf[Class[Writable]],
arr.map(x => anyToWritable(x)).toArray)
}
@@ -489,8 +674,10 @@ object SparkContext {
implicit def writableWritableConverter[T <: Writable]() =
new WritableConverter[T](_.erasure.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
+ /**
+ * 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[_]): Seq[String] = {
val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class")
if (uri != null) {
@@ -505,8 +692,8 @@ object SparkContext {
Nil
}
}
-
- // Find the JAR that contains the class of a particular object
+
+ /** Find the JAR that contains the class of a particular object */
def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass)
}
@@ -518,7 +705,7 @@ object SparkContext {
* 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).
*/
-class WritableConverter[T](
+private[spark] class WritableConverter[T](
val writableClass: ClassManifest[T] => Class[_ <: Writable],
val convert: Writable => T)
extends Serializable
diff --git a/core/src/main/scala/spark/SparkEnv.scala b/core/src/main/scala/spark/SparkEnv.scala
index add8fcec51..4c6ec6cc6e 100644
--- a/core/src/main/scala/spark/SparkEnv.scala
+++ b/core/src/main/scala/spark/SparkEnv.scala
@@ -1,46 +1,57 @@
package spark
import akka.actor.ActorSystem
+import akka.actor.ActorSystemImpl
+import akka.remote.RemoteActorRefProvider
+import serializer.Serializer
import spark.broadcast.BroadcastManager
import spark.storage.BlockManager
import spark.storage.BlockManagerMaster
import spark.network.ConnectionManager
import spark.util.AkkaUtils
+/**
+ * Holds all the runtime environment objects for a running Spark instance (either master or worker),
+ * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently
+ * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these
+ * 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 (
val actorSystem: ActorSystem,
- val cache: Cache,
val serializer: Serializer,
val closureSerializer: Serializer,
val cacheTracker: CacheTracker,
val mapOutputTracker: MapOutputTracker,
val shuffleFetcher: ShuffleFetcher,
- val shuffleManager: ShuffleManager,
val broadcastManager: BroadcastManager,
val blockManager: BlockManager,
- val connectionManager: ConnectionManager
+ val connectionManager: ConnectionManager,
+ val httpFileServer: HttpFileServer
) {
/** No-parameter constructor for unit tests. */
def this() = {
- this(null, null, new JavaSerializer, new JavaSerializer, null, null, null, null, null, null, null)
+ this(null, new JavaSerializer, new JavaSerializer, null, null, null, null, null, null, null)
}
def stop() {
+ httpFileServer.stop()
mapOutputTracker.stop()
cacheTracker.stop()
shuffleFetcher.stop()
- shuffleManager.stop()
broadcastManager.stop()
blockManager.stop()
blockManager.master.stop()
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()
}
}
-object SparkEnv {
+object SparkEnv extends Logging {
private val env = new ThreadLocal[SparkEnv]
def set(e: SparkEnv) {
@@ -66,66 +77,55 @@ object SparkEnv {
System.setProperty("spark.master.port", boundPort.toString)
}
- val serializerClass = System.getProperty("spark.serializer", "spark.KryoSerializer")
- val serializer = Class.forName(serializerClass).newInstance().asInstanceOf[Serializer]
+ val classLoader = Thread.currentThread.getContextClassLoader
+
+ // 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)
+ Class.forName(name, true, classLoader).newInstance().asInstanceOf[T]
+ }
+
+ val serializer = instantiateClass[Serializer]("spark.serializer", "spark.JavaSerializer")
val blockManagerMaster = new BlockManagerMaster(actorSystem, isMaster, isLocal)
-
val blockManager = new BlockManager(blockManagerMaster, serializer)
- val connectionManager = blockManager.connectionManager
-
- val shuffleManager = new ShuffleManager()
+ val connectionManager = blockManager.connectionManager
val broadcastManager = new BroadcastManager(isMaster)
- val closureSerializerClass =
- System.getProperty("spark.closure.serializer", "spark.JavaSerializer")
- val closureSerializer =
- Class.forName(closureSerializerClass).newInstance().asInstanceOf[Serializer]
- val cacheClass = System.getProperty("spark.cache.class", "spark.BoundedMemoryCache")
- val cache = Class.forName(cacheClass).newInstance().asInstanceOf[Cache]
+ val closureSerializer = instantiateClass[Serializer](
+ "spark.closure.serializer", "spark.JavaSerializer")
val cacheTracker = new CacheTracker(actorSystem, isMaster, blockManager)
blockManager.cacheTracker = cacheTracker
val mapOutputTracker = new MapOutputTracker(actorSystem, isMaster)
- val shuffleFetcherClass =
- System.getProperty("spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher")
- val shuffleFetcher =
- Class.forName(shuffleFetcherClass).newInstance().asInstanceOf[ShuffleFetcher]
-
- /*
- if (System.getProperty("spark.stream.distributed", "false") == "true") {
- val blockManagerClass = classOf[spark.storage.BlockManager].asInstanceOf[Class[_]]
- if (isLocal || !isMaster) {
- (new Thread() {
- override def run() {
- println("Wait started")
- Thread.sleep(60000)
- println("Wait ended")
- val receiverClass = Class.forName("spark.stream.TestStreamReceiver4")
- val constructor = receiverClass.getConstructor(blockManagerClass)
- val receiver = constructor.newInstance(blockManager)
- receiver.asInstanceOf[Thread].start()
- }
- }).start()
- }
+ val shuffleFetcher = instantiateClass[ShuffleFetcher](
+ "spark.shuffle.fetcher", "spark.BlockStoreShuffleFetcher")
+
+ val httpFileServer = new HttpFileServer()
+ httpFileServer.initialize()
+ System.setProperty("spark.fileserver.uri", httpFileServer.serverUri)
+
+ // Warn about deprecated spark.cache.class property
+ if (System.getProperty("spark.cache.class") != null) {
+ logWarning("The spark.cache.class property is no longer being used! Specify storage " +
+ "levels using the RDD.persist() method instead.")
}
- */
new SparkEnv(
actorSystem,
- cache,
serializer,
closureSerializer,
cacheTracker,
mapOutputTracker,
shuffleFetcher,
- shuffleManager,
broadcastManager,
blockManager,
- connectionManager)
+ connectionManager,
+ httpFileServer)
}
}
diff --git a/core/src/main/scala/spark/TaskEndReason.scala b/core/src/main/scala/spark/TaskEndReason.scala
index 6e4eb25ed4..420c54bc9a 100644
--- a/core/src/main/scala/spark/TaskEndReason.scala
+++ b/core/src/main/scala/spark/TaskEndReason.scala
@@ -7,10 +7,16 @@ import spark.storage.BlockManagerId
* tasks several times for "ephemeral" failures, and only report back failures that require some
* old stages to be resubmitted, such as shuffle map fetch failures.
*/
-sealed trait TaskEndReason
+private[spark] sealed trait TaskEndReason
-case object Success extends TaskEndReason
+private[spark] case object Success extends TaskEndReason
+
+private[spark]
case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it
+
+private[spark]
case class FetchFailed(bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason
-case class ExceptionFailure(exception: Throwable) extends TaskEndReason
-case class OtherFailure(message: String) extends TaskEndReason
+
+private[spark] case class ExceptionFailure(exception: Throwable) extends TaskEndReason
+
+private[spark] case class OtherFailure(message: String) extends TaskEndReason
diff --git a/core/src/main/scala/spark/TaskState.scala b/core/src/main/scala/spark/TaskState.scala
index 9566b52432..78eb33a628 100644
--- a/core/src/main/scala/spark/TaskState.scala
+++ b/core/src/main/scala/spark/TaskState.scala
@@ -2,7 +2,7 @@ package spark
import org.apache.mesos.Protos.{TaskState => MesosTaskState}
-object TaskState
+private[spark] object TaskState
extends Enumeration("LAUNCHING", "RUNNING", "FINISHED", "FAILED", "KILLED", "LOST") {
val LAUNCHING, RUNNING, FINISHED, FAILED, KILLED, LOST = Value
diff --git a/core/src/main/scala/spark/Utils.scala b/core/src/main/scala/spark/Utils.scala
index 1d33f7d6b3..1bdde25896 100644
--- a/core/src/main/scala/spark/Utils.scala
+++ b/core/src/main/scala/spark/Utils.scala
@@ -1,18 +1,18 @@
package spark
import java.io._
-import java.net.InetAddress
+import java.net.{InetAddress, URL, URI}
+import java.util.{Locale, Random, UUID}
import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
-
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{Path, FileSystem, FileUtil}
import scala.collection.mutable.ArrayBuffer
-import scala.util.Random
-import java.util.{Locale, UUID}
import scala.io.Source
/**
* Various utility methods used by Spark.
*/
-object Utils {
+private object Utils extends Logging {
/** Serialize an object using Java serialization */
def serialize[T](o: T): Array[Byte] = {
val bos = new ByteArrayOutputStream()
@@ -71,7 +71,7 @@ object Utils {
while (dir == null) {
attempts += 1
if (attempts > maxAttempts) {
- throw new IOException("Failed to create a temp directory after " + maxAttempts +
+ throw new IOException("Failed to create a temp directory after " + maxAttempts +
" attempts!")
}
try {
@@ -116,22 +116,84 @@ object Utils {
copyStream(in, out, true)
}
+ /** Download a file from a given URL to the local filesystem */
+ def downloadFile(url: URL, localPath: String) {
+ val in = url.openStream()
+ val out = new FileOutputStream(localPath)
+ Utils.copyStream(in, out, true)
+ }
+
+ /**
+ * Download a file requested by the executor. Supports fetching the file in a variety of ways,
+ * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter.
+ */
+ def fetchFile(url: String, targetDir: File) {
+ val filename = url.split("/").last
+ val targetFile = new File(targetDir, filename)
+ val uri = new URI(url)
+ uri.getScheme match {
+ case "http" | "https" | "ftp" =>
+ logInfo("Fetching " + url + " to " + targetFile)
+ val in = new URL(url).openStream()
+ val out = new FileOutputStream(targetFile)
+ Utils.copyStream(in, out, true)
+ case "file" | null =>
+ // Remove the file if it already exists
+ targetFile.delete()
+ // Symlink the file locally.
+ if (uri.isAbsolute) {
+ // url is absolute, i.e. it starts with "file:///". Extract the source
+ // file's absolute path from the url.
+ val sourceFile = new File(uri)
+ logInfo("Symlinking " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath)
+ FileUtil.symLink(sourceFile.getAbsolutePath, targetFile.getAbsolutePath)
+ } else {
+ // url is not absolute, i.e. itself is the path to the source file.
+ logInfo("Symlinking " + url + " to " + targetFile.getAbsolutePath)
+ FileUtil.symLink(url, targetFile.getAbsolutePath)
+ }
+ case _ =>
+ // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
+ val uri = new URI(url)
+ val conf = new Configuration()
+ val fs = FileSystem.get(uri, conf)
+ val in = fs.open(new Path(uri))
+ val out = new FileOutputStream(targetFile)
+ Utils.copyStream(in, out, true)
+ }
+ // Decompress the file if it's a .tar or .tar.gz
+ if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) {
+ logInfo("Untarring " + filename)
+ Utils.execute(Seq("tar", "-xzf", filename), targetDir)
+ } else if (filename.endsWith(".tar")) {
+ logInfo("Untarring " + filename)
+ Utils.execute(Seq("tar", "-xf", filename), targetDir)
+ }
+ // Make the file executable - That's necessary for scripts
+ FileUtil.chmod(filename, "a+x")
+ }
+
/**
* Shuffle the elements of a collection into a random order, returning the
* 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](seq: TraversableOnce[T]): Seq[T] = {
- val buf = new ArrayBuffer[T]()
- buf ++= seq
- val rand = new Random()
- for (i <- (buf.size - 1) to 1 by -1) {
+ def randomize[T: ClassManifest](seq: TraversableOnce[T]): Seq[T] = {
+ randomizeInPlace(seq.toArray)
+ }
+
+ /**
+ * Shuffle the elements of an array into a random order, modifying the
+ * original array. Returns the original array.
+ */
+ def randomizeInPlace[T](arr: Array[T], rand: Random = new Random): Array[T] = {
+ for (i <- (arr.length - 1) to 1 by -1) {
val j = rand.nextInt(i)
- val tmp = buf(j)
- buf(j) = buf(i)
- buf(i) = tmp
+ val tmp = arr(j)
+ arr(j) = arr(i)
+ arr(i) = tmp
}
- buf
+ arr
}
/**
@@ -155,7 +217,7 @@ object Utils {
def localHostName(): String = {
customHostname.getOrElse(InetAddress.getLocalHost.getHostName)
}
-
+
/**
* Returns a standard ThreadFactory except all threads are daemons.
*/
@@ -179,10 +241,10 @@ object Utils {
return threadPool
}
-
+
/**
- * Return the string to tell how long has passed in seconds. The passing parameter should be in
- * millisecond.
+ * Return the string to tell how long has passed in seconds. The passing parameter should be in
+ * millisecond.
*/
def getUsedTimeMs(startTimeMs: Long): String = {
return " " + (System.currentTimeMillis - startTimeMs) + " ms"
@@ -294,4 +356,43 @@ object Utils {
def execute(command: Seq[String]) {
execute(command, new File("."))
}
+
+
+ /**
+ * When called inside a class in the spark package, returns the name of the user code class
+ * (outside the spark package) that called into Spark, as well as which Spark method they called.
+ * This is used, for example, to tell users where in their code each RDD got created.
+ */
+ def getSparkCallSite: String = {
+ val trace = Thread.currentThread.getStackTrace().filter( el =>
+ (!el.getMethodName.contains("getStackTrace")))
+
+ // Keep crawling up the stack trace until we find the first function not inside of the spark
+ // package. We track the last (shallowest) contiguous Spark method. This might be an RDD
+ // transformation, a SparkContext function (such as parallelize), or anything else that leads
+ // to instantiation of an RDD. We also track the first (deepest) user method, file, and line.
+ var lastSparkMethod = "<unknown>"
+ var firstUserFile = "<unknown>"
+ var firstUserLine = 0
+ var finished = false
+
+ for (el <- trace) {
+ if (!finished) {
+ if (el.getClassName.startsWith("spark.") && !el.getClassName.startsWith("spark.examples.")) {
+ lastSparkMethod = if (el.getMethodName == "<init>") {
+ // Spark method is a constructor; get its class name
+ el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1)
+ } else {
+ el.getMethodName
+ }
+ }
+ else {
+ firstUserLine = el.getLineNumber
+ firstUserFile = el.getFileName
+ finished = true
+ }
+ }
+ }
+ "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine)
+ }
}
diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
index 7c0b17c45e..843e1bd18b 100644
--- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
@@ -22,8 +22,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
import JavaDoubleRDD.fromRDD
+ /** 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.
+ */
def persist(newLevel: StorageLevel): JavaDoubleRDD = fromRDD(srdd.persist(newLevel))
// first() has to be overriden here in order for its return type to be Double instead of Object.
@@ -31,36 +36,63 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
// Transformations (return a new RDD)
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
def distinct(): JavaDoubleRDD = fromRDD(srdd.distinct())
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
+ def distinct(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numSplits))
+
+ /**
+ * Return a new RDD containing only the elements that satisfy a predicate.
+ */
def filter(f: JFunction[Double, java.lang.Boolean]): JavaDoubleRDD =
fromRDD(srdd.filter(x => f(x).booleanValue()))
+ /**
+ * Return a sampled subset of this RDD.
+ */
def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaDoubleRDD =
fromRDD(srdd.sample(withReplacement, fraction, seed))
+ /**
+ * Return the union of this RDD and another one. Any identical elements will appear multiple
+ * times (use `.distinct()` to eliminate them).
+ */
def union(other: JavaDoubleRDD): JavaDoubleRDD = fromRDD(srdd.union(other.srdd))
// Double RDD functions
+ /** Return the sum of the elements in this RDD. */
def sum(): Double = srdd.sum()
+ /** Return a [[spark.StatCounter]] describing the elements in this RDD. */
def stats(): StatCounter = srdd.stats()
+ /** Return the mean of the elements in this RDD. */
def mean(): Double = srdd.mean()
+ /** Return the variance of the elements in this RDD. */
def variance(): Double = srdd.variance()
+ /** Return the standard deviation of the elements in this RDD. */
def stdev(): Double = srdd.stdev()
+ /** Return the approximate mean of the elements in this RDD. */
def meanApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] =
srdd.meanApprox(timeout, confidence)
+ /** Return the approximate mean of the elements in this RDD. */
def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout)
+ /** Return the approximate sum of the elements in this RDD. */
def sumApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] =
srdd.sumApprox(timeout, confidence)
-
+
+ /** Return the approximate sum of the elements in this RDD. */
def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout)
}
diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
index c28a13b061..5c2be534ff 100644
--- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
@@ -1,13 +1,5 @@
package spark.api.java
-import spark.SparkContext.rddToPairRDDFunctions
-import spark.api.java.function.{Function2 => JFunction2}
-import spark.api.java.function.{Function => JFunction}
-import spark.partial.BoundedDouble
-import spark.partial.PartialResult
-import spark.storage.StorageLevel
-import spark._
-
import java.util.{List => JList}
import java.util.Comparator
@@ -19,6 +11,17 @@ import org.apache.hadoop.mapred.OutputFormat
import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat}
import org.apache.hadoop.conf.Configuration
+import spark.api.java.function.{Function2 => JFunction2}
+import spark.api.java.function.{Function => JFunction}
+import spark.partial.BoundedDouble
+import spark.partial.PartialResult
+import spark.OrderedRDDFunctions
+import spark.storage.StorageLevel
+import spark.HashPartitioner
+import spark.Partitioner
+import spark.RDD
+import spark.SparkContext.rddToPairRDDFunctions
+
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]] {
@@ -31,21 +34,44 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
// Common RDD functions
+ /** 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.
+ */
def persist(newLevel: StorageLevel): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd.persist(newLevel))
// Transformations (return a new RDD)
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
def distinct(): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct())
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
+ def distinct(numSplits: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numSplits))
+
+ /**
+ * Return a new RDD containing only the elements that satisfy a predicate.
+ */
def filter(f: Function[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue()))
+ /**
+ * Return a sampled subset of this RDD.
+ */
def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed))
+ /**
+ * Return the union of this RDD and another one. Any identical elements will appear multiple
+ * times (use `.distinct()` to eliminate them).
+ */
def union(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] =
new JavaPairRDD[K, V](rdd.union(other.rdd))
@@ -56,7 +82,21 @@ 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
+ * 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.
+ *
+ * In addition, users can control the partitioning of the output RDD, and whether to perform
+ * map-side aggregation (if a mapper can produce multiple items with the same key).
+ */
def combineByKey[C](createCombiner: Function[V, C],
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C],
@@ -71,50 +111,113 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
))
}
+ /**
+ * Simplified version of combineByKey that hash-partitions the output RDD.
+ */
def combineByKey[C](createCombiner: JFunction[V, C],
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C],
numSplits: Int): JavaPairRDD[K, C] =
combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numSplits))
+ /**
+ * 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.
+ */
def reduceByKey(partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] =
fromRDD(rdd.reduceByKey(partitioner, func))
+ /**
+ * Merge the values for each key using an associative reduce function, but return the results
+ * immediately to the master as a Map. This will also perform the merging locally on each mapper
+ * before sending results to a reducer, similarly to a "combiner" in MapReduce.
+ */
def reduceByKeyLocally(func: JFunction2[V, V, V]): java.util.Map[K, V] =
mapAsJavaMap(rdd.reduceByKeyLocally(func))
+ /** 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.
+ */
def countByKeyApprox(timeout: Long, confidence: Double = 0.95)
: PartialResult[java.util.Map[K, BoundedDouble]] =
rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap)
+ /**
+ * 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 numSplits splits.
+ */
def reduceByKey(func: JFunction2[V, V, V], numSplits: Int): JavaPairRDD[K, V] =
fromRDD(rdd.reduceByKey(func, numSplits))
+ /**
+ * Group the values for each key in the RDD into a single sequence. Allows controlling the
+ * partitioning of the resulting key-value pair RDD by passing a Partitioner.
+ */
def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JList[V]] =
fromRDD(groupByResultToJava(rdd.groupByKey(partitioner)))
+ /**
+ * Group the values for each key in the RDD into a single sequence. Hash-partitions the
+ * resulting RDD with into `numSplits` partitions.
+ */
def groupByKey(numSplits: Int): JavaPairRDD[K, JList[V]] =
fromRDD(groupByResultToJava(rdd.groupByKey(numSplits)))
+ /**
+ * Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine`
+ * is true, Spark will group values of the same key together on the map side before the
+ * repartitioning, to only send each key over the network once. If a large number of
+ * duplicated keys are expected, and the size of the keys are large, `mapSideCombine` should
+ * be set to true.
+ */
def partitionBy(partitioner: Partitioner): JavaPairRDD[K, V] =
fromRDD(rdd.partitionBy(partitioner))
+ /**
+ * 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.
+ */
def join[W](other: JavaPairRDD[K, W], partitioner: Partitioner): JavaPairRDD[K, (V, W)] =
fromRDD(rdd.join(other, partitioner))
+ /**
+ * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
+ * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
+ * pair (k, (v, None)) if no elements in `other` have key k. Uses the given Partitioner to
+ * partition the output RDD.
+ */
def leftOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner)
: JavaPairRDD[K, (V, Option[W])] =
fromRDD(rdd.leftOuterJoin(other, partitioner))
+ /**
+ * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
+ * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
+ * pair (k, (None, w)) if no elements in `this` have key k. Uses the given Partitioner to
+ * partition the output RDD.
+ */
def rightOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner)
: JavaPairRDD[K, (Option[V], W)] =
fromRDD(rdd.rightOuterJoin(other, partitioner))
+ /**
+ * Simplified version of combineByKey that hash-partitions the resulting RDD using the default
+ * parallelism level.
+ */
def combineByKey[C](createCombiner: JFunction[V, C],
mergeValue: JFunction2[C, V, C],
mergeCombiners: JFunction2[C, C, C]): JavaPairRDD[K, C] = {
@@ -123,40 +226,94 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
fromRDD(combineByKey(createCombiner, mergeValue, mergeCombiners))
}
+ /**
+ * 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 the default parallelism level.
+ */
def reduceByKey(func: JFunction2[V, V, V]): JavaPairRDD[K, V] = {
val partitioner = rdd.defaultPartitioner(rdd)
fromRDD(reduceByKey(partitioner, func))
}
+ /**
+ * Group the values for each key in the RDD into a single sequence. Hash-partitions the
+ * resulting RDD with the default parallelism level.
+ */
def groupByKey(): JavaPairRDD[K, JList[V]] =
fromRDD(groupByResultToJava(rdd.groupByKey()))
+ /**
+ * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
+ * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
+ * (k, v2) is in `other`. Performs a hash join across the cluster.
+ */
def join[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, W)] =
fromRDD(rdd.join(other))
+ /**
+ * Return an RDD containing all pairs of elements with matching keys in `this` and `other`. Each
+ * pair of elements will be returned as a (k, (v1, v2)) tuple, where (k, v1) is in `this` and
+ * (k, v2) is in `other`. Performs a hash join across the cluster.
+ */
def join[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (V, W)] =
fromRDD(rdd.join(other, numSplits))
+ /**
+ * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
+ * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
+ * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
+ * using the default level of parallelism.
+ */
def leftOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (V, Option[W])] =
fromRDD(rdd.leftOuterJoin(other))
+ /**
+ * Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
+ * resulting RDD will either contain all pairs (k, (v, Some(w))) for w in `other`, or the
+ * pair (k, (v, None)) if no elements in `other` have key k. Hash-partitions the output
+ * into `numSplits` partitions.
+ */
def leftOuterJoin[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (V, Option[W])] =
fromRDD(rdd.leftOuterJoin(other, numSplits))
+ /**
+ * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
+ * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
+ * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
+ * RDD using the default parallelism level.
+ */
def rightOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Option[V], W)] =
fromRDD(rdd.rightOuterJoin(other))
+ /**
+ * Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
+ * resulting RDD will either contain all pairs (k, (Some(v), w)) for v in `this`, or the
+ * pair (k, (None, w)) if no elements in `this` have key k. Hash-partitions the resulting
+ * RDD into the given number of partitions.
+ */
def rightOuterJoin[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (Option[V], W)] =
fromRDD(rdd.rightOuterJoin(other, numSplits))
+ /**
+ * Return the key-value pairs in this RDD to the master as a Map.
+ */
def collectAsMap(): java.util.Map[K, V] = mapAsJavaMap(rdd.collectAsMap())
+ /**
+ * Pass each value in the key-value pair RDD through a map function without changing the keys;
+ * this also retains the original RDD's partitioning.
+ */
def mapValues[U](f: Function[V, U]): JavaPairRDD[K, U] = {
implicit val cm: ClassManifest[U] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[U]]
fromRDD(rdd.mapValues(f))
}
+ /**
+ * Pass each value in the key-value pair RDD through a flatMap function without changing the
+ * keys; this also retains the original RDD's partitioning.
+ */
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
@@ -165,37 +322,68 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
fromRDD(rdd.flatMapValues(fn))
}
+ /**
+ * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
+ * list of values for that key in `this` as well as `other`.
+ */
def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner)
: JavaPairRDD[K, (JList[V], JList[W])] =
fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner)))
+ /**
+ * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
+ * tuple with the list of values for that key in `this`, `other1` and `other2`.
+ */
def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], partitioner: Partitioner)
: JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] =
fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner)))
+ /**
+ * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
+ * list of values for that key in `this` as well as `other`.
+ */
def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] =
fromRDD(cogroupResultToJava(rdd.cogroup(other)))
+ /**
+ * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
+ * tuple with the list of values for that key in `this`, `other1` and `other2`.
+ */
def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2])
: JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] =
fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2)))
+ /**
+ * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the
+ * list of values for that key in `this` as well as `other`.
+ */
def cogroup[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (JList[V], JList[W])]
= fromRDD(cogroupResultToJava(rdd.cogroup(other, numSplits)))
+ /**
+ * For each key k in `this` or `other1` or `other2`, return a resulting RDD that contains a
+ * tuple with the list of values for that key in `this`, `other1` and `other2`.
+ */
def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numSplits: Int)
: JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] =
fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numSplits)))
+ /** Alias for cogroup. */
def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] =
fromRDD(cogroupResultToJava(rdd.groupWith(other)))
+ /** Alias for cogroup. */
def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2])
: JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] =
fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2)))
+ /**
+ * Return the list of values in the RDD for key `key`. This operation is done efficiently if the
+ * RDD has a known partitioner by only searching the partition that the key maps to.
+ */
def lookup(key: K): JList[V] = seqAsJavaList(rdd.lookup(key))
+ /** Output the RDD to any Hadoop-supported file system. */
def saveAsHadoopFile[F <: OutputFormat[_, _]](
path: String,
keyClass: Class[_],
@@ -205,6 +393,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, conf)
}
+ /** Output the RDD to any Hadoop-supported file system. */
def saveAsHadoopFile[F <: OutputFormat[_, _]](
path: String,
keyClass: Class[_],
@@ -213,6 +402,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass)
}
+ /** Output the RDD to any Hadoop-supported file system. */
def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]](
path: String,
keyClass: Class[_],
@@ -222,6 +412,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
rdd.saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, conf)
}
+ /** Output the RDD to any Hadoop-supported file system. */
def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]](
path: String,
keyClass: Class[_],
@@ -230,21 +421,49 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
rdd.saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass)
}
+ /**
+ * Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for
+ * that storage system. The JobConf should set an OutputFormat and any output paths required
+ * (e.g. a table name to write to) in the same way as it would be configured for a Hadoop
+ * MapReduce job.
+ */
def saveAsHadoopDataset(conf: JobConf) {
rdd.saveAsHadoopDataset(conf)
}
-
- // Ordered RDD Functions
+ /**
+ * Sort the RDD by key, so that each partition contains a sorted range of the elements in
+ * ascending order. 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(): JavaPairRDD[K, V] = sortByKey(true)
+ /**
+ * 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(ascending: Boolean): JavaPairRDD[K, V] = {
val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]]
sortByKey(comp, true)
}
+ /**
+ * 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]): JavaPairRDD[K, V] = sortByKey(comp, true)
+ /**
+ * 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): JavaPairRDD[K, V] = {
class KeyOrdering(val a: K) extends Ordered[K] {
override def compare(b: K) = comp.compare(a, b)
@@ -274,4 +493,4 @@ object JavaPairRDD {
new JavaPairRDD[K, V](rdd)
implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd
-} \ No newline at end of file
+}
diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala
index 541aa1e60b..ac31350ec3 100644
--- a/core/src/main/scala/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDD.scala
@@ -11,20 +11,43 @@ JavaRDDLike[T, JavaRDD[T]] {
// Common RDD functions
+ /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
def cache(): JavaRDD[T] = wrapRDD(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.
+ */
def persist(newLevel: StorageLevel): JavaRDD[T] = wrapRDD(rdd.persist(newLevel))
// Transformations (return a new RDD)
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
def distinct(): JavaRDD[T] = wrapRDD(rdd.distinct())
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
+ def distinct(numSplits: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numSplits))
+
+ /**
+ * Return a new RDD containing only the elements that satisfy a predicate.
+ */
def filter(f: JFunction[T, java.lang.Boolean]): JavaRDD[T] =
wrapRDD(rdd.filter((x => f(x).booleanValue())))
+ /**
+ * Return a sampled subset of this RDD.
+ */
def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] =
wrapRDD(rdd.sample(withReplacement, fraction, seed))
-
+
+ /**
+ * Return the union of this RDD and another one. Any identical elements will appear multiple
+ * times (use `.distinct()` to eliminate them).
+ */
def union(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.union(other.rdd))
}
diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
index 785dd96394..13fcee1004 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
@@ -19,41 +19,71 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
def rdd: RDD[T]
+ /** Set of partitions in this RDD. */
def splits: JList[Split] = new java.util.ArrayList(rdd.splits.toSeq)
+ /** The [[spark.SparkContext]] that this RDD was created on. */
def context: SparkContext = rdd.context
-
+
+ /** A unique ID for this RDD (within its SparkContext). */
def id: Int = rdd.id
+ /** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
def getStorageLevel: StorageLevel = rdd.getStorageLevel
+ /**
+ * Internal method to this RDD; will read from cache if applicable, or otherwise compute it.
+ * This should ''not'' be called by users directly, but is available for implementors of custom
+ * subclasses of RDD.
+ */
def iterator(split: Split): java.util.Iterator[T] = asJavaIterator(rdd.iterator(split))
// Transformations (return a new RDD)
+ /**
+ * Return a new RDD by applying a function to all elements of this RDD.
+ */
def map[R](f: JFunction[T, R]): JavaRDD[R] =
new JavaRDD(rdd.map(f)(f.returnType()))(f.returnType())
+ /**
+ * Return a new RDD by applying a function to all elements of this RDD.
+ */
def map[R](f: DoubleFunction[T]): JavaDoubleRDD =
new JavaDoubleRDD(rdd.map(x => f(x).doubleValue()))
+ /**
+ * 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]]]
new JavaPairRDD(rdd.map(f)(cm))(f.keyType(), f.valueType())
}
+ /**
+ * Return a new RDD by first applying a function to all elements of this
+ * RDD, and then flattening the results.
+ */
def flatMap[U](f: FlatMapFunction[T, U]): JavaRDD[U] = {
import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala
JavaRDD.fromRDD(rdd.flatMap(fn)(f.elementType()))(f.elementType())
}
+ /**
+ * Return a new RDD by first applying a function to all elements of this
+ * RDD, and then flattening the results.
+ */
def flatMap(f: DoubleFlatMapFunction[T]): JavaDoubleRDD = {
import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala
new JavaDoubleRDD(rdd.flatMap(fn).map((x: java.lang.Double) => x.doubleValue()))
}
+ /**
+ * Return a new RDD by first applying a function to all elements of this
+ * RDD, and then flattening the results.
+ */
def flatMap[K, V](f: PairFlatMapFunction[T, K, V]): JavaPairRDD[K, V] = {
import scala.collection.JavaConverters._
def fn = (x: T) => f.apply(x).asScala
@@ -61,29 +91,50 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(f.keyType(), f.valueType())
}
+ /**
+ * Return a new RDD by applying a function to each partition of this RDD.
+ */
def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaRDD[U] = {
def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
JavaRDD.fromRDD(rdd.mapPartitions(fn)(f.elementType()))(f.elementType())
}
+
+ /**
+ * Return a new RDD by applying a function to each partition of this RDD.
+ */
def mapPartitions(f: DoubleFlatMapFunction[java.util.Iterator[T]]): JavaDoubleRDD = {
def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
new JavaDoubleRDD(rdd.mapPartitions(fn).map((x: java.lang.Double) => x.doubleValue()))
}
+ /**
+ * Return a new RDD by applying a function to each partition of this RDD.
+ */
def mapPartitions[K, V](f: PairFlatMapFunction[java.util.Iterator[T], K, V]):
JavaPairRDD[K, V] = {
def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator())
JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(f.keyType(), f.valueType())
}
+ /**
+ * Return an RDD created by coalescing all elements within each partition into an array.
+ */
def glom(): JavaRDD[JList[T]] =
new JavaRDD(rdd.glom().map(x => new java.util.ArrayList[T](x.toSeq)))
+ /**
+ * 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](other: JavaRDDLike[U, _]): JavaPairRDD[T, U] =
JavaPairRDD.fromRDD(rdd.cartesian(other.rdd)(other.classManifest))(classManifest,
other.classManifest)
+ /**
+ * 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]]
@@ -92,6 +143,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType)))(kcm, vcm)
}
+ /**
+ * 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], numSplits: Int): JavaPairRDD[K, JList[T]] = {
implicit val kcm: ClassManifest[K] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
@@ -100,56 +155,114 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numSplits)(f.returnType)))(kcm, vcm)
}
+ /**
+ * Return an RDD created by piping elements to a forked external process.
+ */
def pipe(command: String): JavaRDD[String] = rdd.pipe(command)
+ /**
+ * Return an RDD created by piping elements to a forked external process.
+ */
def pipe(command: JList[String]): JavaRDD[String] =
rdd.pipe(asScalaBuffer(command))
+ /**
+ * Return an RDD created by piping elements to a forked external process.
+ */
def pipe(command: JList[String], env: java.util.Map[String, String]): JavaRDD[String] =
rdd.pipe(asScalaBuffer(command), mapAsScalaMap(env))
// Actions (launch a job to return a value to the user program)
-
+
+ /**
+ * Applies a function f to all elements of this RDD.
+ */
def foreach(f: VoidFunction[T]) {
val cleanF = rdd.context.clean(f)
rdd.foreach(cleanF)
}
+ /**
+ * Return an array that contains all of the elements in this RDD.
+ */
def collect(): JList[T] = {
import scala.collection.JavaConversions._
val arr: java.util.Collection[T] = rdd.collect().toSeq
new java.util.ArrayList(arr)
}
-
+
+ /**
+ * Reduces the elements of this RDD using the specified associative binary operator.
+ */
def reduce(f: JFunction2[T, T, T]): T = rdd.reduce(f)
+ /**
+ * Aggregate the elements of each partition, and then the results for all the partitions, using a
+ * given associative function and a neutral "zero value". The function op(t1, t2) is allowed to
+ * modify t1 and return it as its result value to avoid object allocation; however, it should not
+ * modify t2.
+ */
def fold(zeroValue: T)(f: JFunction2[T, T, T]): T =
rdd.fold(zeroValue)(f)
+ /**
+ * Aggregate the elements of each partition, and then the results for all the partitions, using
+ * given combine functions and a neutral "zero value". This function can return a different result
+ * type, U, than the type of this RDD, T. Thus, we need one operation for merging a T into an U
+ * and one operation for merging two U's, as in scala.TraversableOnce. Both of these functions are
+ * allowed to modify and return their first argument instead of creating a new U to avoid memory
+ * allocation.
+ */
def aggregate[U](zeroValue: U)(seqOp: JFunction2[U, T, U],
combOp: JFunction2[U, U, U]): U =
rdd.aggregate(zeroValue)(seqOp, combOp)(seqOp.returnType)
+ /**
+ * Return the number of elements in the RDD.
+ */
def count(): Long = rdd.count()
+ /**
+ * (Experimental) Approximate version of count() that returns a potentially incomplete result
+ * within a timeout, even if not all tasks have finished.
+ */
def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] =
rdd.countApprox(timeout, confidence)
+ /**
+ * (Experimental) Approximate version of count() that returns a potentially incomplete result
+ * within a timeout, even if not all tasks have finished.
+ */
def countApprox(timeout: Long): PartialResult[BoundedDouble] =
rdd.countApprox(timeout)
+ /**
+ * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final
+ * combine step happens locally on the master, equivalent to running a single reduce task.
+ */
def countByValue(): java.util.Map[T, java.lang.Long] =
mapAsJavaMap(rdd.countByValue().map((x => (x._1, new lang.Long(x._2)))))
+ /**
+ * (Experimental) Approximate version of countByValue().
+ */
def countByValueApprox(
timeout: Long,
confidence: Double
): PartialResult[java.util.Map[T, BoundedDouble]] =
rdd.countByValueApprox(timeout, confidence).map(mapAsJavaMap)
+ /**
+ * (Experimental) Approximate version of countByValue().
+ */
def countByValueApprox(timeout: Long): PartialResult[java.util.Map[T, BoundedDouble]] =
rdd.countByValueApprox(timeout).map(mapAsJavaMap)
+ /**
+ * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so
+ * it will be slow if a lot of partitions are required. In that case, use collect() to get the
+ * whole RDD instead.
+ */
def take(num: Int): JList[T] = {
import scala.collection.JavaConversions._
val arr: java.util.Collection[T] = rdd.take(num).toSeq
@@ -162,9 +275,18 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
new java.util.ArrayList(arr)
}
+ /**
+ * Return the first element in this RDD.
+ */
def first(): T = rdd.first()
+ /**
+ * Save this RDD as a text file, using string representations of elements.
+ */
def saveAsTextFile(path: String) = rdd.saveAsTextFile(path)
+ /**
+ * Save this RDD as a SequenceFile of serialized objects.
+ */
def saveAsObjectFile(path: String) = rdd.saveAsObjectFile(path)
}
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
index 4a7d945a8d..edbb187b1b 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
@@ -1,43 +1,78 @@
package spark.api.java
-import spark.{Accumulator, AccumulatorParam, RDD, SparkContext}
-import spark.SparkContext.IntAccumulatorParam
-import spark.SparkContext.DoubleAccumulatorParam
-import spark.broadcast.Broadcast
+import java.util.{Map => JMap}
+import scala.collection.JavaConversions
import scala.collection.JavaConversions._
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.mapred.InputFormat
import org.apache.hadoop.mapred.JobConf
-
import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat}
+import spark.{Accumulator, AccumulatorParam, RDD, SparkContext}
+import spark.SparkContext.IntAccumulatorParam
+import spark.SparkContext.DoubleAccumulatorParam
+import spark.broadcast.Broadcast
-import scala.collection.JavaConversions
-
+/**
+ * A Java-friendly version of [[spark.SparkContext]] that returns [[spark.api.java.JavaRDD]]s and
+ * works with Java collections instead of Scala ones.
+ */
class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
- def this(master: String, frameworkName: String) = this(new SparkContext(master, frameworkName))
+ /**
+ * @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
+ */
+ def this(master: String, jobName: String) = this(new SparkContext(master, jobName))
+
+ /**
+ * @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 sparkHome The SPARK_HOME directory on the slave 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.
+ */
+ def this(master: String, jobName: String, sparkHome: String, jarFile: String) =
+ this(new SparkContext(master, jobName, sparkHome, Seq(jarFile)))
- def this(master: String, frameworkName: String, sparkHome: String, jarFile: String) =
- this(new SparkContext(master, frameworkName, sparkHome, Seq(jarFile)))
+ /**
+ * @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 sparkHome The SPARK_HOME directory on the slave 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.
+ */
+ def this(master: String, jobName: String, sparkHome: String, jars: Array[String]) =
+ this(new SparkContext(master, jobName, sparkHome, jars.toSeq))
- def this(master: String, frameworkName: String, sparkHome: String, jars: Array[String]) =
- this(new SparkContext(master, frameworkName, sparkHome, jars.toSeq))
+ /**
+ * @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 sparkHome The SPARK_HOME directory on the slave 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, jobName: String, sparkHome: String, jars: Array[String],
+ environment: JMap[String, String]) =
+ this(new SparkContext(master, jobName, sparkHome, jars.toSeq, environment))
- val env = sc.env
+ private[spark] val env = sc.env
+ /** 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]]
sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices)
}
+ /** Distribute a local Scala collection to form an RDD. */
def parallelize[T](list: java.util.List[T]): JavaRDD[T] =
parallelize(list, sc.defaultParallelism)
-
+ /** 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] =
@@ -47,21 +82,32 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
JavaPairRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices))
}
+ /** Distribute a local Scala collection to form an RDD. */
def parallelizePairs[K, V](list: java.util.List[Tuple2[K, V]]): JavaPairRDD[K, V] =
parallelizePairs(list, sc.defaultParallelism)
+ /** Distribute a local Scala collection to form an RDD. */
def parallelizeDoubles(list: java.util.List[java.lang.Double], numSlices: Int): JavaDoubleRDD =
JavaDoubleRDD.fromRDD(sc.parallelize(JavaConversions.asScalaBuffer(list).map(_.doubleValue()),
numSlices))
+ /** Distribute a local Scala collection to form an RDD. */
def parallelizeDoubles(list: java.util.List[java.lang.Double]): JavaDoubleRDD =
parallelizeDoubles(list, sc.defaultParallelism)
+ /**
+ * Read a text file from HDFS, a local file system (available on all nodes), or any
+ * Hadoop-supported file system URI, and return it as an RDD of Strings.
+ */
def textFile(path: String): JavaRDD[String] = sc.textFile(path)
+ /**
+ * Read a text file from HDFS, a local file system (available on all nodes), or any
+ * Hadoop-supported file system URI, and return it as an RDD of Strings.
+ */
def textFile(path: String, minSplits: Int): JavaRDD[String] = sc.textFile(path, minSplits)
- /**Get an RDD for a Hadoop SequenceFile with given key and value types */
+ /**Get an RDD for a Hadoop SequenceFile with given key and value types. */
def sequenceFile[K, V](path: String,
keyClass: Class[K],
valueClass: Class[V],
@@ -72,6 +118,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
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)
@@ -92,6 +139,13 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
sc.objectFile(path, minSplits)(cm)
}
+ /**
+ * Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and
+ * BytesWritable values that contain a serialized partition. This is still an experimental storage
+ * format and may not be supported exactly as is in future Spark releases. It will also be pretty
+ * 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](path: String): JavaRDD[T] = {
implicit val cm: ClassManifest[T] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[T]]
@@ -115,6 +169,11 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits))
}
+ /**
+ * Get an RDD for a Hadoop-readable dataset from a Hadooop JobConf giving its InputFormat and any
+ * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
+ * etc).
+ */
def hadoopRDD[K, V, F <: InputFormat[K, V]](
conf: JobConf,
inputFormatClass: Class[F],
@@ -126,7 +185,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass))
}
- /**Get an RDD for a Hadoop file with an arbitrary InputFormat */
+ /** Get an RDD for a Hadoop file with an arbitrary InputFormat */
def hadoopFile[K, V, F <: InputFormat[K, V]](
path: String,
inputFormatClass: Class[F],
@@ -139,6 +198,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits))
}
+ /** Get an RDD for a Hadoop file with an arbitrary InputFormat */
def hadoopFile[K, V, F <: InputFormat[K, V]](
path: String,
inputFormatClass: Class[F],
@@ -180,12 +240,14 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
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
sc.union(rdds)(cm)
}
+ /** Build the union of two or more RDDs. */
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)
@@ -195,26 +257,49 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.union(rdds)(cm))(kcm, vcm)
}
+ /** Build the union of two or more RDDs. */
override def union(first: JavaDoubleRDD, rest: java.util.List[JavaDoubleRDD]): JavaDoubleRDD = {
val rdds: Seq[RDD[Double]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.srdd)
new JavaDoubleRDD(sc.union(rdds))
}
+ /**
+ * Create an [[spark.Accumulator]] integer variable, which tasks can "add" values
+ * to using the `+=` method. Only the master can access the accumulator's `value`.
+ */
def intAccumulator(initialValue: Int): Accumulator[Int] =
sc.accumulator(initialValue)(IntAccumulatorParam)
+ /**
+ * Create an [[spark.Accumulator]] double variable, which tasks can "add" values
+ * to using the `+=` method. Only the master can access the accumulator's `value`.
+ */
def doubleAccumulator(initialValue: Double): Accumulator[Double] =
sc.accumulator(initialValue)(DoubleAccumulatorParam)
+ /**
+ * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values
+ * to using the `+=` method. Only the master can access the accumulator's `value`.
+ */
def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] =
sc.accumulator(initialValue)(accumulatorParam)
+ /**
+ * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for
+ * reading it in distributed functions. The variable will be sent to each cluster only once.
+ */
def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value)
+ /** Shut down the SparkContext. */
def stop() {
sc.stop()
}
+ /**
+ * Get Spark's home location from either a value set through the constructor,
+ * or the spark.home Java property, or the SPARK_HOME environment variable
+ * (in that order of preference). If neither of these is set, return None.
+ */
def getSparkHome(): Option[String] = sc.getSparkHome()
}
diff --git a/core/src/main/scala/spark/api/java/StorageLevels.java b/core/src/main/scala/spark/api/java/StorageLevels.java
new file mode 100644
index 0000000000..722af3c06c
--- /dev/null
+++ b/core/src/main/scala/spark/api/java/StorageLevels.java
@@ -0,0 +1,20 @@
+package spark.api.java;
+
+import spark.storage.StorageLevel;
+
+/**
+ * Expose some commonly useful storage level constants.
+ */
+public class StorageLevels {
+ public static final StorageLevel NONE = new StorageLevel(false, false, false, 1);
+ public static final StorageLevel DISK_ONLY = new StorageLevel(true, false, false, 1);
+ public static final StorageLevel DISK_ONLY_2 = new StorageLevel(true, false, false, 2);
+ public static final StorageLevel MEMORY_ONLY = new StorageLevel(false, true, true, 1);
+ public static final StorageLevel MEMORY_ONLY_2 = new StorageLevel(false, true, true, 2);
+ public static final StorageLevel MEMORY_ONLY_SER = new StorageLevel(false, true, false, 1);
+ public static final StorageLevel MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, 2);
+ public static final StorageLevel MEMORY_AND_DISK = new StorageLevel(true, true, true, 1);
+ public static final StorageLevel MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2);
+ public static final StorageLevel MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, 1);
+ public static final StorageLevel MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2);
+}
diff --git a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
index 7b6478c2cd..3a8192be3a 100644
--- a/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
+++ b/core/src/main/scala/spark/api/java/function/DoubleFlatMapFunction.java
@@ -5,6 +5,9 @@ import scala.runtime.AbstractFunction1;
import java.io.Serializable;
+/**
+ * A function that returns zero or more records of type Double from each input record.
+ */
// DoubleFlatMapFunction does not extend FlatMapFunction because flatMap is
// overloaded for both FlatMapFunction and DoubleFlatMapFunction.
public abstract class DoubleFlatMapFunction<T> extends AbstractFunction1<T, Iterable<Double>>
diff --git a/core/src/main/scala/spark/api/java/function/DoubleFunction.java b/core/src/main/scala/spark/api/java/function/DoubleFunction.java
index a03a72c835..c6ef76d088 100644
--- a/core/src/main/scala/spark/api/java/function/DoubleFunction.java
+++ b/core/src/main/scala/spark/api/java/function/DoubleFunction.java
@@ -5,6 +5,9 @@ import scala.runtime.AbstractFunction1;
import java.io.Serializable;
+/**
+ * A function that returns Doubles, and can be used to construct DoubleRDDs.
+ */
// DoubleFunction does not extend Function because some UDF functions, like map,
// are overloaded for both Function and DoubleFunction.
public abstract class DoubleFunction<T> extends WrappedFunction1<T, Double>
diff --git a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
index bcba38c569..e027cdacd3 100644
--- a/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
+++ b/core/src/main/scala/spark/api/java/function/FlatMapFunction.scala
@@ -1,5 +1,8 @@
package spark.api.java.function
+/**
+ * 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]] {
@throws(classOf[Exception])
def call(x: T) : java.lang.Iterable[R]
diff --git a/core/src/main/scala/spark/api/java/function/Function.java b/core/src/main/scala/spark/api/java/function/Function.java
index f6f2e5fd76..dae8295f21 100644
--- a/core/src/main/scala/spark/api/java/function/Function.java
+++ b/core/src/main/scala/spark/api/java/function/Function.java
@@ -8,8 +8,9 @@ import java.io.Serializable;
/**
- * Base class for functions whose return types do not have special RDDs; DoubleFunction is
- * handled separately, to allow DoubleRDDs to be constructed when mapping RDDs to doubles.
+ * Base class for functions whose return types do not create special RDDs. PairFunction and
+ * DoubleFunction are handled separately, to allow PairRDDs and DoubleRDDs to be constructed
+ * when mapping RDDs of other types.
*/
public abstract class Function<T, R> extends WrappedFunction1<T, R> implements Serializable {
public abstract R call(T t) throws Exception;
diff --git a/core/src/main/scala/spark/api/java/function/Function2.java b/core/src/main/scala/spark/api/java/function/Function2.java
index be48b173b8..69bf12c8c9 100644
--- a/core/src/main/scala/spark/api/java/function/Function2.java
+++ b/core/src/main/scala/spark/api/java/function/Function2.java
@@ -6,6 +6,9 @@ import scala.runtime.AbstractFunction2;
import java.io.Serializable;
+/**
+ * A two-argument function that takes arguments of type T1 and T2 and returns an R.
+ */
public abstract class Function2<T1, T2, R> extends WrappedFunction2<T1, T2, R>
implements Serializable {
diff --git a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
index c074b9c717..b3cc4df6aa 100644
--- a/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
+++ b/core/src/main/scala/spark/api/java/function/PairFlatMapFunction.java
@@ -7,6 +7,10 @@ import scala.runtime.AbstractFunction1;
import java.io.Serializable;
+/**
+ * A function that returns zero or more key-value pair records from each input record. The
+ * key-value pairs are represented as scala.Tuple2 objects.
+ */
// PairFlatMapFunction does not extend FlatMapFunction because flatMap is
// overloaded for both FlatMapFunction and PairFlatMapFunction.
public abstract class PairFlatMapFunction<T, K, V>
diff --git a/core/src/main/scala/spark/api/java/function/PairFunction.java b/core/src/main/scala/spark/api/java/function/PairFunction.java
index 7f5bb7de13..9fc6df4b88 100644
--- a/core/src/main/scala/spark/api/java/function/PairFunction.java
+++ b/core/src/main/scala/spark/api/java/function/PairFunction.java
@@ -7,6 +7,9 @@ import scala.runtime.AbstractFunction1;
import java.io.Serializable;
+/**
+ * A function that returns key-value pairs (Tuple2<K, V>), and can be used to construct PairRDDs.
+ */
// PairFunction does not extend Function because some UDF functions, like map,
// are overloaded for both Function and PairFunction.
public abstract class PairFunction<T, K, V>
diff --git a/core/src/main/scala/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/spark/api/java/function/VoidFunction.scala
index 0eefe337e8..b0096cf2bf 100644
--- a/core/src/main/scala/spark/api/java/function/VoidFunction.scala
+++ b/core/src/main/scala/spark/api/java/function/VoidFunction.scala
@@ -1,5 +1,8 @@
package spark.api.java.function
+/**
+ * A function with no return value.
+ */
// This allows Java users to write void methods without having to return Unit.
abstract class VoidFunction[T] extends Serializable {
@throws(classOf[Exception])
diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
index d08e1e9fbf..923f5cdf4f 100644
--- a/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
+++ b/core/src/main/scala/spark/api/java/function/WrappedFunction1.scala
@@ -7,7 +7,7 @@ import scala.runtime.AbstractFunction1
* apply() method as call() and declare that it can throw Exception (since AbstractFunction1.apply
* isn't marked to allow that).
*/
-abstract class WrappedFunction1[T, R] extends AbstractFunction1[T, R] {
+private[spark] abstract class WrappedFunction1[T, R] extends AbstractFunction1[T, R] {
@throws(classOf[Exception])
def call(t: T): R
diff --git a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
index c9d67d9771..2c6e9b1571 100644
--- a/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
+++ b/core/src/main/scala/spark/api/java/function/WrappedFunction2.scala
@@ -7,7 +7,7 @@ import scala.runtime.AbstractFunction2
* apply() method as call() and declare that it can throw Exception (since AbstractFunction2.apply
* isn't marked to allow that).
*/
-abstract class WrappedFunction2[T1, T2, R] extends AbstractFunction2[T1, T2, R] {
+private[spark] abstract class WrappedFunction2[T1, T2, R] extends AbstractFunction2[T1, T2, R] {
@throws(classOf[Exception])
def call(t1: T1, t2: T2): R
diff --git a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
index 016dc00fb0..ef27bbb502 100644
--- a/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/BitTorrentBroadcast.scala
@@ -11,14 +11,17 @@ import scala.math
import spark._
import spark.storage.StorageLevel
-class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean)
-extends Broadcast[T] with Logging with Serializable {
+private[spark] class BitTorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
+ extends Broadcast[T](id)
+ with Logging
+ with Serializable {
def value = value_
+ def blockId: String = "broadcast_" + id
+
MultiTracker.synchronized {
- SparkEnv.get.blockManager.putSingle(
- uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false)
+ SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
}
@transient var arrayOfBlocks: Array[BroadcastBlock] = null
@@ -45,7 +48,7 @@ extends Broadcast[T] with Logging with Serializable {
// Used only in Workers
@transient var ttGuide: TalkToGuide = null
- @transient var hostAddress = Utils.localIpAddress
+ @transient var hostAddress = Utils.localIpAddress()
@transient var listenPort = -1
@transient var guidePort = -1
@@ -53,7 +56,7 @@ extends Broadcast[T] with Logging with Serializable {
// Must call this after all the variables have been created/initialized
if (!isLocal) {
- sendBroadcast
+ sendBroadcast()
}
def sendBroadcast() {
@@ -106,20 +109,22 @@ extends Broadcast[T] with Logging with Serializable {
listOfSources += masterSource
// Register with the Tracker
- MultiTracker.registerBroadcast(uuid,
+ MultiTracker.registerBroadcast(id,
SourceInfo(hostAddress, guidePort, totalBlocks, totalBytes))
}
private def readObject(in: ObjectInputStream) {
in.defaultReadObject()
MultiTracker.synchronized {
- SparkEnv.get.blockManager.getSingle(uuid.toString) match {
- case Some(x) => x.asInstanceOf[T]
- case None => {
- logInfo("Started reading broadcast variable " + uuid)
+ SparkEnv.get.blockManager.getSingle(blockId) match {
+ case Some(x) =>
+ value_ = x.asInstanceOf[T]
+
+ case None =>
+ logInfo("Started reading broadcast variable " + id)
// Initializing everything because Master will only send null/0 values
// Only the 1st worker in a node can be here. Others will get from cache
- initializeWorkerVariables
+ initializeWorkerVariables()
logInfo("Local host address: " + hostAddress)
@@ -131,18 +136,17 @@ extends Broadcast[T] with Logging with Serializable {
val start = System.nanoTime
- val receptionSucceeded = receiveBroadcast(uuid)
+ val receptionSucceeded = receiveBroadcast(id)
if (receptionSucceeded) {
value_ = MultiTracker.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
SparkEnv.get.blockManager.putSingle(
- uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false)
+ blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
} else {
- logError("Reading Broadcasted variable " + uuid + " failed")
+ logError("Reading broadcast variable " + id + " failed")
}
val time = (System.nanoTime - start) / 1e9
- logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s")
- }
+ logInfo("Reading broadcast variable " + id + " took " + time + " s")
}
}
}
@@ -254,8 +258,8 @@ extends Broadcast[T] with Logging with Serializable {
}
}
- def receiveBroadcast(variableUUID: UUID): Boolean = {
- val gInfo = MultiTracker.getGuideInfo(variableUUID)
+ def receiveBroadcast(variableID: Long): Boolean = {
+ val gInfo = MultiTracker.getGuideInfo(variableID)
if (gInfo.listenPort == SourceInfo.TxOverGoToDefault) {
return false
@@ -307,9 +311,11 @@ extends Broadcast[T] with Logging with Serializable {
var threadPool = Utils.newDaemonFixedThreadPool(MultiTracker.MaxChatSlots)
while (hasBlocks.get < totalBlocks) {
- var numThreadsToCreate =
- math.min(listOfSources.size, MultiTracker.MaxChatSlots) -
+ var numThreadsToCreate = 0
+ listOfSources.synchronized {
+ numThreadsToCreate = math.min(listOfSources.size, MultiTracker.MaxChatSlots) -
threadPool.getActiveCount
+ }
while (hasBlocks.get < totalBlocks && numThreadsToCreate > 0) {
var peerToTalkTo = pickPeerToTalkToRandom
@@ -722,7 +728,6 @@ extends Broadcast[T] with Logging with Serializable {
guidePortLock.synchronized { guidePortLock.notifyAll() }
try {
- // Don't stop until there is a copy in HDFS
while (!stopBroadcast) {
var clientSocket: Socket = null
try {
@@ -730,14 +735,17 @@ extends Broadcast[T] with Logging with Serializable {
clientSocket = serverSocket.accept()
} catch {
case e: Exception => {
- logError("GuideMultipleRequests Timeout.")
-
// Stop broadcast if at least one worker has connected and
// everyone connected so far are done. Comparing with
// listOfSources.size - 1, because it includes the Guide itself
- if (listOfSources.size > 1 &&
- setOfCompletedSources.size == listOfSources.size - 1) {
- stopBroadcast = true
+ listOfSources.synchronized {
+ setOfCompletedSources.synchronized {
+ if (listOfSources.size > 1 &&
+ setOfCompletedSources.size == listOfSources.size - 1) {
+ stopBroadcast = true
+ logInfo("GuideMultipleRequests Timeout. stopBroadcast == true.")
+ }
+ }
}
}
}
@@ -760,7 +768,7 @@ extends Broadcast[T] with Logging with Serializable {
logInfo("Sending stopBroadcast notifications...")
sendStopBroadcastNotifications
- MultiTracker.unregisterBroadcast(uuid)
+ MultiTracker.unregisterBroadcast(id)
} finally {
if (serverSocket != null) {
logInfo("GuideMultipleRequests now stopping...")
@@ -918,9 +926,7 @@ extends Broadcast[T] with Logging with Serializable {
serverSocket.setSoTimeout(MultiTracker.ServerSocketTimeout)
clientSocket = serverSocket.accept()
} catch {
- case e: Exception => {
- logError("ServeMultipleRequests Timeout.")
- }
+ case e: Exception => { }
}
if (clientSocket != null) {
logDebug("Serve: Accepted new client connection:" + clientSocket)
@@ -1023,9 +1029,12 @@ extends Broadcast[T] with Logging with Serializable {
}
}
-class BitTorrentBroadcastFactory
+private[spark] class BitTorrentBroadcastFactory
extends BroadcastFactory {
- def initialize(isMaster: Boolean) = MultiTracker.initialize(isMaster)
- def newBroadcast[T](value_ : T, isLocal: Boolean) = new BitTorrentBroadcast[T](value_, isLocal)
- def stop() = MultiTracker.stop
+ def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) }
+
+ def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
+ new BitTorrentBroadcast[T](value_, isLocal, id)
+
+ def stop() { MultiTracker.stop() }
}
diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala
index d68e56a114..6055bfd045 100644
--- a/core/src/main/scala/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/spark/broadcast/Broadcast.scala
@@ -1,25 +1,20 @@
package spark.broadcast
import java.io._
-import java.net._
-import java.util.{BitSet, UUID}
-import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
-
-import scala.collection.mutable.Map
+import java.util.concurrent.atomic.AtomicLong
import spark._
-trait Broadcast[T] extends Serializable {
- val uuid = UUID.randomUUID
-
+abstract class Broadcast[T](id: Long) extends Serializable {
def value: T
// We cannot have an abstract readObject here due to some weird issues with
// readObject having to be 'private' in sub-classes.
- override def toString = "spark.Broadcast(" + uuid + ")"
+ override def toString = "spark.Broadcast(" + id + ")"
}
+private[spark]
class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializable {
private var initialized = false
@@ -49,14 +44,10 @@ class BroadcastManager(val isMaster_ : Boolean) extends Logging with Serializabl
broadcastFactory.stop()
}
- private def getBroadcastFactory: BroadcastFactory = {
- if (broadcastFactory == null) {
- throw new SparkException ("Broadcast.getBroadcastFactory called before initialize")
- }
- broadcastFactory
- }
+ private val nextBroadcastId = new AtomicLong(0)
- def newBroadcast[T](value_ : T, isLocal: Boolean) = broadcastFactory.newBroadcast[T](value_, isLocal)
+ def newBroadcast[T](value_ : T, isLocal: Boolean) =
+ broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement())
def isMaster = isMaster_
}
diff --git a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
index e341d556bf..ab6d302827 100644
--- a/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
+++ b/core/src/main/scala/spark/broadcast/BroadcastFactory.scala
@@ -6,8 +6,8 @@ package spark.broadcast
* BroadcastFactory implementation to instantiate a particular broadcast for the
* entire Spark job.
*/
-trait BroadcastFactory {
+private[spark] trait BroadcastFactory {
def initialize(isMaster: Boolean): Unit
- def newBroadcast[T](value_ : T, isLocal: Boolean): Broadcast[T]
+ def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long): Broadcast[T]
def stop(): Unit
}
diff --git a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
index 03986ea756..7eb4ddb74f 100644
--- a/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/HttpBroadcast.scala
@@ -12,44 +12,47 @@ import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
import spark._
import spark.storage.StorageLevel
-class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean)
-extends Broadcast[T] with Logging with Serializable {
+private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
+extends Broadcast[T](id) with Logging with Serializable {
def value = value_
+ def blockId: String = "broadcast_" + id
+
HttpBroadcast.synchronized {
- SparkEnv.get.blockManager.putSingle(
- uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false)
+ SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
}
if (!isLocal) {
- HttpBroadcast.write(uuid, value_)
+ HttpBroadcast.write(id, value_)
}
// Called by JVM when deserializing an object
private def readObject(in: ObjectInputStream) {
in.defaultReadObject()
HttpBroadcast.synchronized {
- SparkEnv.get.blockManager.getSingle(uuid.toString) match {
+ SparkEnv.get.blockManager.getSingle(blockId) match {
case Some(x) => value_ = x.asInstanceOf[T]
case None => {
- logInfo("Started reading broadcast variable " + uuid)
+ logInfo("Started reading broadcast variable " + id)
val start = System.nanoTime
- value_ = HttpBroadcast.read[T](uuid)
- SparkEnv.get.blockManager.putSingle(
- uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false)
+ value_ = HttpBroadcast.read[T](id)
+ SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
val time = (System.nanoTime - start) / 1e9
- logInfo("Reading broadcast variable " + uuid + " took " + time + " s")
+ logInfo("Reading broadcast variable " + id + " took " + time + " s")
}
}
}
}
}
-class HttpBroadcastFactory extends BroadcastFactory {
- def initialize(isMaster: Boolean) = HttpBroadcast.initialize(isMaster)
- def newBroadcast[T](value_ : T, isLocal: Boolean) = new HttpBroadcast[T](value_, isLocal)
- def stop() = HttpBroadcast.stop()
+private[spark] class HttpBroadcastFactory extends BroadcastFactory {
+ def initialize(isMaster: Boolean) { HttpBroadcast.initialize(isMaster) }
+
+ def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
+ new HttpBroadcast[T](value_, isLocal, id)
+
+ def stop() { HttpBroadcast.stop() }
}
private object HttpBroadcast extends Logging {
@@ -65,7 +68,7 @@ private object HttpBroadcast extends Logging {
synchronized {
if (!initialized) {
bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
- compress = System.getProperty("spark.compress", "false").toBoolean
+ compress = System.getProperty("spark.broadcast.compress", "true").toBoolean
if (isMaster) {
createServer()
}
@@ -76,9 +79,12 @@ private object HttpBroadcast extends Logging {
}
def stop() {
- if (server != null) {
- server.stop()
- server = null
+ synchronized {
+ if (server != null) {
+ server.stop()
+ server = null
+ }
+ initialized = false
}
}
@@ -91,8 +97,8 @@ private object HttpBroadcast extends Logging {
logInfo("Broadcast server started at " + serverUri)
}
- def write(uuid: UUID, value: Any) {
- val file = new File(broadcastDir, "broadcast-" + uuid)
+ def write(id: Long, value: Any) {
+ val file = new File(broadcastDir, "broadcast-" + id)
val out: OutputStream = if (compress) {
new LZFOutputStream(new FileOutputStream(file)) // Does its own buffering
} else {
@@ -104,8 +110,8 @@ private object HttpBroadcast extends Logging {
serOut.close()
}
- def read[T](uuid: UUID): T = {
- val url = serverUri + "/broadcast-" + uuid
+ def read[T](id: Long): T = {
+ val url = serverUri + "/broadcast-" + id
var in = if (compress) {
new LZFInputStream(new URL(url).openStream()) // Does its own buffering
} else {
diff --git a/core/src/main/scala/spark/broadcast/MultiTracker.scala b/core/src/main/scala/spark/broadcast/MultiTracker.scala
index d5f5b22461..5e76dedb94 100644
--- a/core/src/main/scala/spark/broadcast/MultiTracker.scala
+++ b/core/src/main/scala/spark/broadcast/MultiTracker.scala
@@ -2,8 +2,7 @@ package spark.broadcast
import java.io._
import java.net._
-import java.util.{UUID, Random}
-import java.util.concurrent.{Executors, ThreadFactory, ThreadPoolExecutor}
+import java.util.Random
import scala.collection.mutable.Map
@@ -18,7 +17,7 @@ extends Logging {
val FIND_BROADCAST_TRACKER = 2
// Map to keep track of guides of ongoing broadcasts
- var valueToGuideMap = Map[UUID, SourceInfo]()
+ var valueToGuideMap = Map[Long, SourceInfo]()
// Random number generator
var ranGen = new Random
@@ -154,44 +153,44 @@ extends Logging {
val messageType = ois.readObject.asInstanceOf[Int]
if (messageType == REGISTER_BROADCAST_TRACKER) {
- // Receive UUID
- val uuid = ois.readObject.asInstanceOf[UUID]
+ // Receive Long
+ val id = ois.readObject.asInstanceOf[Long]
// Receive hostAddress and listenPort
val gInfo = ois.readObject.asInstanceOf[SourceInfo]
// Add to the map
valueToGuideMap.synchronized {
- valueToGuideMap += (uuid -> gInfo)
+ valueToGuideMap += (id -> gInfo)
}
- logInfo ("New broadcast " + uuid + " registered with TrackMultipleValues. Ongoing ones: " + valueToGuideMap)
+ logInfo ("New broadcast " + id + " registered with TrackMultipleValues. Ongoing ones: " + valueToGuideMap)
// Send dummy ACK
oos.writeObject(-1)
oos.flush()
} else if (messageType == UNREGISTER_BROADCAST_TRACKER) {
- // Receive UUID
- val uuid = ois.readObject.asInstanceOf[UUID]
+ // Receive Long
+ val id = ois.readObject.asInstanceOf[Long]
// Remove from the map
valueToGuideMap.synchronized {
- valueToGuideMap(uuid) = SourceInfo("", SourceInfo.TxOverGoToDefault)
+ valueToGuideMap(id) = SourceInfo("", SourceInfo.TxOverGoToDefault)
}
- logInfo ("Broadcast " + uuid + " unregistered from TrackMultipleValues. Ongoing ones: " + valueToGuideMap)
+ logInfo ("Broadcast " + id + " unregistered from TrackMultipleValues. Ongoing ones: " + valueToGuideMap)
// Send dummy ACK
oos.writeObject(-1)
oos.flush()
} else if (messageType == FIND_BROADCAST_TRACKER) {
- // Receive UUID
- val uuid = ois.readObject.asInstanceOf[UUID]
+ // Receive Long
+ val id = ois.readObject.asInstanceOf[Long]
var gInfo =
- if (valueToGuideMap.contains(uuid)) valueToGuideMap(uuid)
+ if (valueToGuideMap.contains(id)) valueToGuideMap(id)
else SourceInfo("", SourceInfo.TxNotStartedRetry)
- logDebug("Got new request: " + clientSocket + " for " + uuid + " : " + gInfo.listenPort)
+ logDebug("Got new request: " + clientSocket + " for " + id + " : " + gInfo.listenPort)
// Send reply back
oos.writeObject(gInfo)
@@ -224,12 +223,12 @@ extends Logging {
}
}
- def getGuideInfo(variableUUID: UUID): SourceInfo = {
+ def getGuideInfo(variableLong: Long): SourceInfo = {
var clientSocketToTracker: Socket = null
var oosTracker: ObjectOutputStream = null
var oisTracker: ObjectInputStream = null
- var gInfo: SourceInfo = SourceInfo("", SourceInfo.TxOverGoToDefault)
+ var gInfo: SourceInfo = SourceInfo("", SourceInfo.TxNotStartedRetry)
var retriesLeft = MultiTracker.MaxRetryCount
do {
@@ -247,8 +246,8 @@ extends Logging {
oosTracker.writeObject(MultiTracker.FIND_BROADCAST_TRACKER)
oosTracker.flush()
- // Send UUID and receive GuideInfo
- oosTracker.writeObject(variableUUID)
+ // Send Long and receive GuideInfo
+ oosTracker.writeObject(variableLong)
oosTracker.flush()
gInfo = oisTracker.readObject.asInstanceOf[SourceInfo]
} catch {
@@ -276,7 +275,7 @@ extends Logging {
return gInfo
}
- def registerBroadcast(uuid: UUID, gInfo: SourceInfo) {
+ def registerBroadcast(id: Long, gInfo: SourceInfo) {
val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort)
val oosST = new ObjectOutputStream(socket.getOutputStream)
oosST.flush()
@@ -286,8 +285,8 @@ extends Logging {
oosST.writeObject(REGISTER_BROADCAST_TRACKER)
oosST.flush()
- // Send UUID of this broadcast
- oosST.writeObject(uuid)
+ // Send Long of this broadcast
+ oosST.writeObject(id)
oosST.flush()
// Send this tracker's information
@@ -303,7 +302,7 @@ extends Logging {
socket.close()
}
- def unregisterBroadcast(uuid: UUID) {
+ def unregisterBroadcast(id: Long) {
val socket = new Socket(MultiTracker.MasterHostAddress, MasterTrackerPort)
val oosST = new ObjectOutputStream(socket.getOutputStream)
oosST.flush()
@@ -313,8 +312,8 @@ extends Logging {
oosST.writeObject(UNREGISTER_BROADCAST_TRACKER)
oosST.flush()
- // Send UUID of this broadcast
- oosST.writeObject(uuid)
+ // Send Long of this broadcast
+ oosST.writeObject(id)
oosST.flush()
// Receive ACK and throw it away
@@ -383,10 +382,10 @@ extends Logging {
}
}
-case class BroadcastBlock(blockID: Int, byteArray: Array[Byte])
+private[spark] case class BroadcastBlock(blockID: Int, byteArray: Array[Byte])
extends Serializable
-case class VariableInfo(@transient arrayOfBlocks : Array[BroadcastBlock],
+private[spark] case class VariableInfo(@transient arrayOfBlocks : Array[BroadcastBlock],
totalBlocks: Int,
totalBytes: Int)
extends Serializable {
diff --git a/core/src/main/scala/spark/broadcast/SourceInfo.scala b/core/src/main/scala/spark/broadcast/SourceInfo.scala
index f90385fd47..c79bb93c38 100644
--- a/core/src/main/scala/spark/broadcast/SourceInfo.scala
+++ b/core/src/main/scala/spark/broadcast/SourceInfo.scala
@@ -7,7 +7,7 @@ import spark._
/**
* Used to keep and pass around information of peers involved in a broadcast
*/
-case class SourceInfo (hostAddress: String,
+private[spark] case class SourceInfo (hostAddress: String,
listenPort: Int,
totalBlocks: Int = SourceInfo.UnusedParam,
totalBytes: Int = SourceInfo.UnusedParam)
@@ -26,10 +26,11 @@ extends Comparable[SourceInfo] with Logging {
/**
* Helper Object of SourceInfo for its constants
*/
-object SourceInfo {
- // Constants for special values of listenPort
+private[spark] object SourceInfo {
+ // Broadcast has not started yet! Should never happen.
val TxNotStartedRetry = -1
- val TxOverGoToDefault = 0
+ // Broadcast has already finished. Try default mechanism.
+ val TxOverGoToDefault = -3
// Other constants
val StopBroadcast = -2
val UnusedParam = 0
diff --git a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
index c9e1e67d87..fa676e9064 100644
--- a/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
+++ b/core/src/main/scala/spark/broadcast/TreeBroadcast.scala
@@ -10,14 +10,15 @@ import scala.math
import spark._
import spark.storage.StorageLevel
-class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean)
-extends Broadcast[T] with Logging with Serializable {
+private[spark] class TreeBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long)
+extends Broadcast[T](id) with Logging with Serializable {
def value = value_
+ def blockId = "broadcast_" + id
+
MultiTracker.synchronized {
- SparkEnv.get.blockManager.putSingle(
- uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false)
+ SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
}
@transient var arrayOfBlocks: Array[BroadcastBlock] = null
@@ -35,7 +36,7 @@ extends Broadcast[T] with Logging with Serializable {
@transient var serveMR: ServeMultipleRequests = null
@transient var guideMR: GuideMultipleRequests = null
- @transient var hostAddress = Utils.localIpAddress
+ @transient var hostAddress = Utils.localIpAddress()
@transient var listenPort = -1
@transient var guidePort = -1
@@ -43,7 +44,7 @@ extends Broadcast[T] with Logging with Serializable {
// Must call this after all the variables have been created/initialized
if (!isLocal) {
- sendBroadcast
+ sendBroadcast()
}
def sendBroadcast() {
@@ -84,20 +85,22 @@ extends Broadcast[T] with Logging with Serializable {
listOfSources += masterSource
// Register with the Tracker
- MultiTracker.registerBroadcast(uuid,
+ MultiTracker.registerBroadcast(id,
SourceInfo(hostAddress, guidePort, totalBlocks, totalBytes))
}
private def readObject(in: ObjectInputStream) {
in.defaultReadObject()
MultiTracker.synchronized {
- SparkEnv.get.blockManager.getSingle(uuid.toString) match {
- case Some(x) => x.asInstanceOf[T]
- case None => {
- logInfo("Started reading broadcast variable " + uuid)
+ SparkEnv.get.blockManager.getSingle(blockId) match {
+ case Some(x) =>
+ value_ = x.asInstanceOf[T]
+
+ case None =>
+ logInfo("Started reading broadcast variable " + id)
// Initializing everything because Master will only send null/0 values
// Only the 1st worker in a node can be here. Others will get from cache
- initializeWorkerVariables
+ initializeWorkerVariables()
logInfo("Local host address: " + hostAddress)
@@ -108,18 +111,17 @@ extends Broadcast[T] with Logging with Serializable {
val start = System.nanoTime
- val receptionSucceeded = receiveBroadcast(uuid)
+ val receptionSucceeded = receiveBroadcast(id)
if (receptionSucceeded) {
value_ = MultiTracker.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks)
SparkEnv.get.blockManager.putSingle(
- uuid.toString, value_, StorageLevel.MEMORY_ONLY_DESER, false)
+ blockId, value_, StorageLevel.MEMORY_AND_DISK, false)
} else {
- logError("Reading Broadcasted variable " + uuid + " failed")
+ logError("Reading broadcast variable " + id + " failed")
}
val time = (System.nanoTime - start) / 1e9
- logInfo("Reading Broadcasted variable " + uuid + " took " + time + " s")
- }
+ logInfo("Reading broadcast variable " + id + " took " + time + " s")
}
}
}
@@ -136,14 +138,14 @@ extends Broadcast[T] with Logging with Serializable {
serveMR = null
- hostAddress = Utils.localIpAddress
+ hostAddress = Utils.localIpAddress()
listenPort = -1
stopBroadcast = false
}
- def receiveBroadcast(variableUUID: UUID): Boolean = {
- val gInfo = MultiTracker.getGuideInfo(variableUUID)
+ def receiveBroadcast(variableID: Long): Boolean = {
+ val gInfo = MultiTracker.getGuideInfo(variableID)
if (gInfo.listenPort == SourceInfo.TxOverGoToDefault) {
return false
@@ -290,15 +292,17 @@ extends Broadcast[T] with Logging with Serializable {
clientSocket = serverSocket.accept
} catch {
case e: Exception => {
- logError("GuideMultipleRequests Timeout.")
-
// Stop broadcast if at least one worker has connected and
- // everyone connected so far are done.
- // Comparing with listOfSources.size - 1, because the Guide itself
- // is included
- if (listOfSources.size > 1 &&
- setOfCompletedSources.size == listOfSources.size - 1) {
- stopBroadcast = true
+ // everyone connected so far are done. Comparing with
+ // listOfSources.size - 1, because it includes the Guide itself
+ listOfSources.synchronized {
+ setOfCompletedSources.synchronized {
+ if (listOfSources.size > 1 &&
+ setOfCompletedSources.size == listOfSources.size - 1) {
+ stopBroadcast = true
+ logInfo("GuideMultipleRequests Timeout. stopBroadcast == true.")
+ }
+ }
}
}
}
@@ -316,7 +320,7 @@ extends Broadcast[T] with Logging with Serializable {
logInfo("Sending stopBroadcast notifications...")
sendStopBroadcastNotifications
- MultiTracker.unregisterBroadcast(uuid)
+ MultiTracker.unregisterBroadcast(id)
} finally {
if (serverSocket != null) {
logInfo("GuideMultipleRequests now stopping...")
@@ -490,7 +494,7 @@ extends Broadcast[T] with Logging with Serializable {
serverSocket.setSoTimeout(MultiTracker.ServerSocketTimeout)
clientSocket = serverSocket.accept
} catch {
- case e: Exception => logError("ServeMultipleRequests Timeout.")
+ case e: Exception => { }
}
if (clientSocket != null) {
@@ -570,9 +574,12 @@ extends Broadcast[T] with Logging with Serializable {
}
}
-class TreeBroadcastFactory
+private[spark] class TreeBroadcastFactory
extends BroadcastFactory {
- def initialize(isMaster: Boolean) = MultiTracker.initialize(isMaster)
- def newBroadcast[T](value_ : T, isLocal: Boolean) = new TreeBroadcast[T](value_, isLocal)
- def stop() = MultiTracker.stop
+ def initialize(isMaster: Boolean) { MultiTracker.initialize(isMaster) }
+
+ def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) =
+ new TreeBroadcast[T](value_, isLocal, id)
+
+ def stop() { MultiTracker.stop() }
}
diff --git a/core/src/main/scala/spark/deploy/Command.scala b/core/src/main/scala/spark/deploy/Command.scala
index 344888919a..577101e3c3 100644
--- a/core/src/main/scala/spark/deploy/Command.scala
+++ b/core/src/main/scala/spark/deploy/Command.scala
@@ -2,7 +2,7 @@ package spark.deploy
import scala.collection.Map
-case class Command(
+private[spark] case class Command(
mainClass: String,
arguments: Seq[String],
environment: Map[String, String]) {
diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala
index 141bbe4d57..d2b63d6e0d 100644
--- a/core/src/main/scala/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/spark/deploy/DeployMessage.scala
@@ -7,13 +7,15 @@ import scala.collection.immutable.List
import scala.collection.mutable.HashMap
-sealed trait DeployMessage extends Serializable
+private[spark] sealed trait DeployMessage extends Serializable
// Worker to Master
+private[spark]
case class RegisterWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int)
extends DeployMessage
+private[spark]
case class ExecutorStateChanged(
jobId: String,
execId: Int,
@@ -23,11 +25,11 @@ case class ExecutorStateChanged(
// Master to Worker
-case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
-case class RegisterWorkerFailed(message: String) extends DeployMessage
-case class KillExecutor(jobId: String, execId: Int) extends DeployMessage
+private[spark] case class RegisteredWorker(masterWebUiUrl: String) extends DeployMessage
+private[spark] case class RegisterWorkerFailed(message: String) extends DeployMessage
+private[spark] case class KillExecutor(jobId: String, execId: Int) extends DeployMessage
-case class LaunchExecutor(
+private[spark] case class LaunchExecutor(
jobId: String,
execId: Int,
jobDesc: JobDescription,
@@ -38,33 +40,42 @@ case class LaunchExecutor(
// Client to Master
-case class RegisterJob(jobDescription: JobDescription) extends DeployMessage
+private[spark] case class RegisterJob(jobDescription: JobDescription) extends DeployMessage
// Master to Client
+private[spark]
case class RegisteredJob(jobId: String) extends DeployMessage
+
+private[spark]
case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int)
+
+private[spark]
case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String])
+
+private[spark]
case class JobKilled(message: String)
// Internal message in Client
-case object StopClient
+private[spark] case object StopClient
// MasterWebUI To Master
-case object RequestMasterState
+private[spark] case object RequestMasterState
// Master to MasterWebUI
+private[spark]
case class MasterState(uri : String, workers: List[WorkerInfo], activeJobs: List[JobInfo],
completedJobs: List[JobInfo])
// WorkerWebUI to Worker
-case object RequestWorkerState
+private[spark] case object RequestWorkerState
// Worker to WorkerWebUI
+private[spark]
case class WorkerState(uri: String, workerId: String, executors: List[ExecutorRunner],
finishedExecutors: List[ExecutorRunner], masterUrl: String, cores: Int, memory: Int,
coresUsed: Int, memoryUsed: Int, masterWebUiUrl: String) \ No newline at end of file
diff --git a/core/src/main/scala/spark/deploy/ExecutorState.scala b/core/src/main/scala/spark/deploy/ExecutorState.scala
index d6ff1c54ca..5dc0c54552 100644
--- a/core/src/main/scala/spark/deploy/ExecutorState.scala
+++ b/core/src/main/scala/spark/deploy/ExecutorState.scala
@@ -1,6 +1,6 @@
package spark.deploy
-object ExecutorState
+private[spark] object ExecutorState
extends Enumeration("LAUNCHING", "LOADING", "RUNNING", "KILLED", "FAILED", "LOST") {
val LAUNCHING, LOADING, RUNNING, KILLED, FAILED, LOST = Value
diff --git a/core/src/main/scala/spark/deploy/JobDescription.scala b/core/src/main/scala/spark/deploy/JobDescription.scala
index 8ae77b1038..20879c5f11 100644
--- a/core/src/main/scala/spark/deploy/JobDescription.scala
+++ b/core/src/main/scala/spark/deploy/JobDescription.scala
@@ -1,6 +1,6 @@
package spark.deploy
-class JobDescription(
+private[spark] class JobDescription(
val name: String,
val cores: Int,
val memoryPerSlave: Int,
diff --git a/core/src/main/scala/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
new file mode 100644
index 0000000000..8b2a71add5
--- /dev/null
+++ b/core/src/main/scala/spark/deploy/LocalSparkCluster.scala
@@ -0,0 +1,58 @@
+package spark.deploy
+
+import akka.actor.{ActorRef, Props, Actor, ActorSystem, Terminated}
+
+import spark.deploy.worker.Worker
+import spark.deploy.master.Master
+import spark.util.AkkaUtils
+import spark.{Logging, Utils}
+
+import scala.collection.mutable.ArrayBuffer
+
+private[spark]
+class LocalSparkCluster(numSlaves: Int, coresPerSlave: Int, memoryPerSlave: Int) extends Logging {
+
+ val localIpAddress = Utils.localIpAddress
+
+ var masterActor : ActorRef = _
+ var masterActorSystem : ActorSystem = _
+ var masterPort : Int = _
+ var masterUrl : String = _
+
+ val slaveActorSystems = ArrayBuffer[ActorSystem]()
+ val slaveActors = ArrayBuffer[ActorRef]()
+
+ def start() : String = {
+ logInfo("Starting a local Spark cluster with " + numSlaves + " slaves.")
+
+ /* Start the Master */
+ val (actorSystem, masterPort) = AkkaUtils.createActorSystem("sparkMaster", localIpAddress, 0)
+ masterActorSystem = actorSystem
+ masterUrl = "spark://" + localIpAddress + ":" + masterPort
+ val actor = masterActorSystem.actorOf(
+ Props(new Master(localIpAddress, masterPort, 0)), name = "Master")
+ masterActor = actor
+
+ /* Start the Slaves */
+ for (slaveNum <- 1 to numSlaves) {
+ val (actorSystem, boundPort) =
+ AkkaUtils.createActorSystem("sparkWorker" + slaveNum, localIpAddress, 0)
+ slaveActorSystems += actorSystem
+ val actor = actorSystem.actorOf(
+ Props(new Worker(localIpAddress, boundPort, 0, coresPerSlave, memoryPerSlave, masterUrl)),
+ name = "Worker")
+ slaveActors += actor
+ }
+
+ return masterUrl
+ }
+
+ def stop() {
+ logInfo("Shutting down local Spark cluster.")
+ // Stop the slaves before the master so they don't get upset that it disconnected
+ slaveActorSystems.foreach(_.shutdown())
+ slaveActorSystems.foreach(_.awaitTermination())
+ masterActorSystem.shutdown()
+ masterActorSystem.awaitTermination()
+ }
+}
diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala
index c7fa8a3874..e51b0c5c15 100644
--- a/core/src/main/scala/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/spark/deploy/client/Client.scala
@@ -4,6 +4,7 @@ import spark.deploy._
import akka.actor._
import akka.pattern.ask
import akka.util.duration._
+import akka.pattern.AskTimeoutException
import spark.{SparkException, Logging}
import akka.remote.RemoteClientLifeCycleEvent
import akka.remote.RemoteClientShutdown
@@ -16,7 +17,7 @@ import akka.dispatch.Await
* The main class used to talk to a Spark deploy cluster. Takes a master URL, a job description,
* and a listener for job events, and calls back the listener when various events occur.
*/
-class Client(
+private[spark] class Client(
actorSystem: ActorSystem,
masterUrl: String,
jobDescription: JobDescription,
@@ -42,7 +43,6 @@ class Client(
val akkaUrl = "akka://spark@%s:%s/user/Master".format(masterHost, masterPort)
try {
master = context.actorFor(akkaUrl)
- //master ! RegisterWorker(ip, port, cores, memory)
master ! RegisterJob(jobDescription)
context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
context.watch(master) // Doesn't work with remote actors, but useful for testing
@@ -101,9 +101,13 @@ class Client(
def stop() {
if (actor != null) {
- val timeout = 1.seconds
- val future = actor.ask(StopClient)(timeout)
- Await.result(future, timeout)
+ try {
+ val timeout = 1.seconds
+ val future = actor.ask(StopClient)(timeout)
+ Await.result(future, timeout)
+ } catch {
+ case e: AskTimeoutException => // Ignore it, maybe master went away
+ }
actor = null
}
}
diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala
index 7d23baff32..a8fa982085 100644
--- a/core/src/main/scala/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala
@@ -7,7 +7,7 @@ package spark.deploy.client
*
* Users of this API should *not* block inside the callback methods.
*/
-trait ClientListener {
+private[spark] trait ClientListener {
def connected(jobId: String): Unit
def disconnected(): Unit
diff --git a/core/src/main/scala/spark/deploy/client/TestClient.scala b/core/src/main/scala/spark/deploy/client/TestClient.scala
index df9a36c7fe..bf0e7428ba 100644
--- a/core/src/main/scala/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/spark/deploy/client/TestClient.scala
@@ -4,7 +4,7 @@ import spark.util.AkkaUtils
import spark.{Logging, Utils}
import spark.deploy.{Command, JobDescription}
-object TestClient {
+private[spark] object TestClient {
class TestListener extends ClientListener with Logging {
def connected(id: String) {
diff --git a/core/src/main/scala/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/spark/deploy/client/TestExecutor.scala
index 2e40e10d18..0e46db2272 100644
--- a/core/src/main/scala/spark/deploy/client/TestExecutor.scala
+++ b/core/src/main/scala/spark/deploy/client/TestExecutor.scala
@@ -1,6 +1,6 @@
package spark.deploy.client
-object TestExecutor {
+private[spark] object TestExecutor {
def main(args: Array[String]) {
println("Hello world!")
while (true) {
diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
index 335e00958c..1db2c32633 100644
--- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
@@ -2,7 +2,7 @@ package spark.deploy.master
import spark.deploy.ExecutorState
-class ExecutorInfo(
+private[spark] class ExecutorInfo(
val id: Int,
val job: JobInfo,
val worker: WorkerInfo,
diff --git a/core/src/main/scala/spark/deploy/master/JobInfo.scala b/core/src/main/scala/spark/deploy/master/JobInfo.scala
index 31d48b82b9..8795c09cc1 100644
--- a/core/src/main/scala/spark/deploy/master/JobInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/JobInfo.scala
@@ -5,6 +5,7 @@ import java.util.Date
import akka.actor.ActorRef
import scala.collection.mutable
+private[spark]
class JobInfo(val id: String, val desc: JobDescription, val submitDate: Date, val actor: ActorRef) {
var state = JobState.WAITING
var executors = new mutable.HashMap[Int, ExecutorInfo]
@@ -31,4 +32,13 @@ class JobInfo(val id: String, val desc: JobDescription, val submitDate: Date, va
}
def coresLeft: Int = desc.cores - coresGranted
+
+ private var _retryCount = 0
+
+ def retryCount = _retryCount
+
+ def incrementRetryCount = {
+ _retryCount += 1
+ _retryCount
+ }
}
diff --git a/core/src/main/scala/spark/deploy/master/JobState.scala b/core/src/main/scala/spark/deploy/master/JobState.scala
index 50b0c6f95b..2b70cf0191 100644
--- a/core/src/main/scala/spark/deploy/master/JobState.scala
+++ b/core/src/main/scala/spark/deploy/master/JobState.scala
@@ -1,7 +1,9 @@
package spark.deploy.master
-object JobState extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
+private[spark] object JobState extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
type JobState = Value
val WAITING, RUNNING, FINISHED, FAILED = Value
+
+ val MAX_NUM_RETRY = 10
}
diff --git a/core/src/main/scala/spark/deploy/master/Master.scala b/core/src/main/scala/spark/deploy/master/Master.scala
index c98dddea7b..6010f7cff2 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -1,21 +1,20 @@
package spark.deploy.master
-import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
-
import akka.actor._
-import spark.{Logging, Utils}
-import spark.util.AkkaUtils
+import akka.actor.Terminated
+import akka.remote.{RemoteClientLifeCycleEvent, RemoteClientDisconnected, RemoteClientShutdown}
+
import java.text.SimpleDateFormat
import java.util.Date
-import akka.remote.RemoteClientLifeCycleEvent
+
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
+
import spark.deploy._
-import akka.remote.RemoteClientShutdown
-import akka.remote.RemoteClientDisconnected
-import spark.deploy.RegisterWorker
-import spark.deploy.RegisterWorkerFailed
-import akka.actor.Terminated
+import spark.{Logging, SparkException, Utils}
+import spark.util.AkkaUtils
-class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
+
+private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For job IDs
var nextJobNumber = 0
@@ -81,12 +80,22 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
exec.state = state
exec.job.actor ! ExecutorUpdated(execId, state, message)
if (ExecutorState.isFinished(state)) {
+ val jobInfo = idToJob(jobId)
// Remove this executor from the worker and job
logInfo("Removing executor " + exec.fullId + " because it is " + state)
- idToJob(jobId).removeExecutor(exec)
+ jobInfo.removeExecutor(exec)
exec.worker.removeExecutor(exec)
- // TODO: the worker would probably want to restart the executor a few times
- schedule()
+
+ // Only retry certain number of times so we don't go into an infinite loop.
+ if (jobInfo.incrementRetryCount <= JobState.MAX_NUM_RETRY) {
+ schedule()
+ } else {
+ val e = new SparkException("Job %s wth ID %s failed %d times.".format(
+ jobInfo.desc.name, jobInfo.id, jobInfo.retryCount))
+ logError(e.getMessage, e)
+ throw e
+ //System.exit(1)
+ }
}
}
case None =>
@@ -112,7 +121,7 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
addressToWorker.get(address).foreach(removeWorker)
addressToJob.get(address).foreach(removeJob)
}
-
+
case RequestMasterState => {
sender ! MasterState(ip + ":" + port, workers.toList, jobs.toList, completedJobs.toList)
}
@@ -203,7 +212,7 @@ class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
}
}
-object Master {
+private[spark] object Master {
def main(argStrings: Array[String]) {
val args = new MasterArguments(argStrings)
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port)
diff --git a/core/src/main/scala/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/spark/deploy/master/MasterArguments.scala
index 0f7a92bdd0..1b1c3dd0ad 100644
--- a/core/src/main/scala/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/spark/deploy/master/MasterArguments.scala
@@ -6,7 +6,7 @@ import spark.Utils
/**
* Command-line parser for the master.
*/
-class MasterArguments(args: Array[String]) {
+private[spark] class MasterArguments(args: Array[String]) {
var ip = Utils.localIpAddress()
var port = 7077
var webUiPort = 8080
@@ -51,7 +51,7 @@ class MasterArguments(args: Array[String]) {
*/
def printUsageAndExit(exitCode: Int) {
System.err.println(
- "Usage: spark-master [options]\n" +
+ "Usage: Master [options]\n" +
"\n" +
"Options:\n" +
" -i IP, --ip IP IP address or DNS name to listen on\n" +
diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
index f03c0a0229..700a41c770 100644
--- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
+++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
@@ -10,6 +10,7 @@ import cc.spray.directives._
import cc.spray.typeconversion.TwirlSupport._
import spark.deploy._
+private[spark]
class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives {
val RESOURCE_DIR = "spark/deploy/master/webui"
val STATIC_RESOURCE_DIR = "spark/deploy/static"
@@ -22,7 +23,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct
completeWith {
val future = master ? RequestMasterState
future.map {
- masterState => masterui.html.index.render(masterState.asInstanceOf[MasterState])
+ masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState])
}
}
} ~
@@ -36,7 +37,7 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct
// A bit ugly an inefficient, but we won't have a number of jobs
// so large that it will make a significant difference.
(masterState.activeJobs ::: masterState.completedJobs).find(_.id == jobId) match {
- case Some(job) => masterui.html.job_details.render(job)
+ case Some(job) => spark.deploy.master.html.job_details.render(job)
case _ => null
}
}
diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
index 59474a0945..16b3f9b653 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
@@ -3,7 +3,7 @@ package spark.deploy.master
import akka.actor.ActorRef
import scala.collection.mutable
-class WorkerInfo(
+private[spark] class WorkerInfo(
val id: String,
val host: String,
val port: Int,
diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
index 3e24380810..07ae7bca78 100644
--- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
@@ -13,7 +13,7 @@ import spark.deploy.ExecutorStateChanged
/**
* Manages the execution of one executor process.
*/
-class ExecutorRunner(
+private[spark] class ExecutorRunner(
val jobId: String,
val execId: Int,
val jobDesc: JobDescription,
@@ -29,12 +29,25 @@ class ExecutorRunner(
val fullId = jobId + "/" + execId
var workerThread: Thread = null
var process: Process = null
+ var shutdownHook: Thread = null
def start() {
workerThread = new Thread("ExecutorRunner for " + fullId) {
override def run() { fetchAndRunExecutor() }
}
workerThread.start()
+
+ // Shutdown hook that kills actors on shutdown.
+ shutdownHook = new Thread() {
+ override def run() {
+ if (process != null) {
+ logInfo("Shutdown hook killing child process.")
+ process.destroy()
+ process.waitFor()
+ }
+ }
+ }
+ Runtime.getRuntime.addShutdownHook(shutdownHook)
}
/** Stop this executor runner, including killing the process it launched */
@@ -45,40 +58,10 @@ class ExecutorRunner(
if (process != null) {
logInfo("Killing process!")
process.destroy()
+ process.waitFor()
}
worker ! ExecutorStateChanged(jobId, execId, ExecutorState.KILLED, None)
- }
- }
-
- /**
- * Download a file requested by the executor. Supports fetching the file in a variety of ways,
- * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter.
- */
- def fetchFile(url: String, targetDir: File) {
- val filename = url.split("/").last
- val targetFile = new File(targetDir, filename)
- if (url.startsWith("http://") || url.startsWith("https://") || url.startsWith("ftp://")) {
- // Use the java.net library to fetch it
- logInfo("Fetching " + url + " to " + targetFile)
- val in = new URL(url).openStream()
- val out = new FileOutputStream(targetFile)
- Utils.copyStream(in, out, true)
- } else {
- // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others
- val uri = new URI(url)
- val conf = new Configuration()
- val fs = FileSystem.get(uri, conf)
- val in = fs.open(new Path(uri))
- val out = new FileOutputStream(targetFile)
- Utils.copyStream(in, out, true)
- }
- // Decompress the file if it's a .tar or .tar.gz
- if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) {
- logInfo("Untarring " + filename)
- Utils.execute(Seq("tar", "-xzf", filename), targetDir)
- } else if (filename.endsWith(".tar")) {
- logInfo("Untarring " + filename)
- Utils.execute(Seq("tar", "-xf", filename), targetDir)
+ Runtime.getRuntime.removeShutdownHook(shutdownHook)
}
}
@@ -92,7 +75,8 @@ class ExecutorRunner(
def buildCommandSeq(): Seq[String] = {
val command = jobDesc.command
- val runScript = new File(sparkHome, "run").getCanonicalPath
+ val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run";
+ val runScript = new File(sparkHome, script).getCanonicalPath
Seq(runScript, command.mainClass) ++ command.arguments.map(substituteVariables)
}
@@ -101,7 +85,12 @@ class ExecutorRunner(
val out = new FileOutputStream(file)
new Thread("redirect output to " + file) {
override def run() {
- Utils.copyStream(in, out, true)
+ try {
+ Utils.copyStream(in, out, true)
+ } catch {
+ case e: IOException =>
+ logInfo("Redirection to " + file + " closed: " + e.getMessage)
+ }
}
}.start()
}
@@ -131,6 +120,9 @@ class ExecutorRunner(
}
env.put("SPARK_CORES", cores.toString)
env.put("SPARK_MEMORY", memory.toString)
+ // In case we are running this from within the Spark Shell, avoid creating a "scala"
+ // parent process for the executor command
+ env.put("SPARK_LAUNCH_WITH_SCALA", "0")
process = builder.start()
// Redirect its stdout and stderr to files
diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala
index 0a80463c0b..474c9364fd 100644
--- a/core/src/main/scala/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/spark/deploy/worker/Worker.scala
@@ -16,7 +16,14 @@ import spark.deploy.RegisterWorkerFailed
import akka.actor.Terminated
import java.io.File
-class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, masterUrl: String)
+private[spark] class Worker(
+ ip: String,
+ port: Int,
+ webUiPort: Int,
+ cores: Int,
+ memory: Int,
+ masterUrl: String,
+ workDirPath: String = null)
extends Actor with Logging {
val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs
@@ -37,7 +44,11 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas
def memoryFree: Int = memory - memoryUsed
def createWorkDir() {
- workDir = new File(sparkHome, "work")
+ workDir = if (workDirPath != null) {
+ new File(workDirPath)
+ } else {
+ new File(sparkHome, "work")
+ }
try {
if (!workDir.exists() && !workDir.mkdirs()) {
logError("Failed to create work directory " + workDir)
@@ -153,14 +164,19 @@ class Worker(ip: String, port: Int, webUiPort: Int, cores: Int, memory: Int, mas
def generateWorkerId(): String = {
"worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), ip, port)
}
+
+ override def postStop() {
+ executors.values.foreach(_.kill())
+ }
}
-object Worker {
+private[spark] object Worker {
def main(argStrings: Array[String]) {
val args = new WorkerArguments(argStrings)
val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", args.ip, args.port)
val actor = actorSystem.actorOf(
- Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory, args.master)),
+ Props(new Worker(args.ip, boundPort, args.webUiPort, args.cores, args.memory,
+ args.master, args.workDir)),
name = "Worker")
actorSystem.awaitTermination()
}
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
index 1efe8304ea..60dc107a4c 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
@@ -8,13 +8,14 @@ import java.lang.management.ManagementFactory
/**
* Command-line parser for the master.
*/
-class WorkerArguments(args: Array[String]) {
+private[spark] class WorkerArguments(args: Array[String]) {
var ip = Utils.localIpAddress()
var port = 0
var webUiPort = 8081
var cores = inferDefaultCores()
var memory = inferDefaultMemory()
var master: String = null
+ var workDir: String = null
// Check for settings in environment variables
if (System.getenv("SPARK_WORKER_PORT") != null) {
@@ -29,6 +30,9 @@ class WorkerArguments(args: Array[String]) {
if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) {
webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt
}
+ if (System.getenv("SPARK_WORKER_DIR") != null) {
+ workDir = System.getenv("SPARK_WORKER_DIR")
+ }
parse(args.toList)
@@ -49,6 +53,10 @@ class WorkerArguments(args: Array[String]) {
memory = value
parse(tail)
+ case ("--work-dir" | "-d") :: value :: tail =>
+ workDir = value
+ parse(tail)
+
case "--webui-port" :: IntParam(value) :: tail =>
webUiPort = value
parse(tail)
@@ -77,13 +85,14 @@ class WorkerArguments(args: Array[String]) {
*/
def printUsageAndExit(exitCode: Int) {
System.err.println(
- "Usage: spark-worker [options] <master>\n" +
+ "Usage: Worker [options] <master>\n" +
"\n" +
"Master must be a URL of the form spark://hostname:port\n" +
"\n" +
"Options:\n" +
" -c CORES, --cores CORES Number of cores to use\n" +
" -m MEM, --memory MEM Amount of memory to use (e.g. 1000M, 2G)\n" +
+ " -d DIR, --work-dir DIR Directory to run jobs in (default: SPARK_HOME/work)\n" +
" -i IP, --ip IP IP address or DNS name to listen on\n" +
" -p PORT, --port PORT Port to listen on (default: random)\n" +
" --webui-port PORT Port for web UI (default: 8081)")
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
index 58a05e1a38..d06f4884ee 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
@@ -9,6 +9,7 @@ import cc.spray.Directives
import cc.spray.typeconversion.TwirlSupport._
import spark.deploy.{WorkerState, RequestWorkerState}
+private[spark]
class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives {
val RESOURCE_DIR = "spark/deploy/worker/webui"
val STATIC_RESOURCE_DIR = "spark/deploy/static"
@@ -21,7 +22,7 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct
completeWith{
val future = worker ? RequestWorkerState
future.map { workerState =>
- workerui.html.index(workerState.asInstanceOf[WorkerState])
+ spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState])
}
}
} ~
diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala
index dba209ac27..dfdb22024e 100644
--- a/core/src/main/scala/spark/executor/Executor.scala
+++ b/core/src/main/scala/spark/executor/Executor.scala
@@ -1,10 +1,12 @@
package spark.executor
import java.io.{File, FileOutputStream}
-import java.net.{URL, URLClassLoader}
+import java.net.{URI, URL, URLClassLoader}
import java.util.concurrent._
-import scala.collection.mutable.ArrayBuffer
+import org.apache.hadoop.fs.FileUtil
+
+import scala.collection.mutable.{ArrayBuffer, Map, HashMap}
import spark.broadcast._
import spark.scheduler._
@@ -14,11 +16,16 @@ import java.nio.ByteBuffer
/**
* The Mesos executor for Spark.
*/
-class Executor extends Logging {
- var classLoader: ClassLoader = null
+private[spark] class Executor extends Logging {
+ var urlClassLoader : ExecutorURLClassLoader = null
var threadPool: ExecutorService = null
var env: SparkEnv = 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]()
+
val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0))
initLogging()
@@ -32,14 +39,14 @@ class Executor extends Logging {
System.setProperty(key, value)
}
+ // Create our ClassLoader and set it on this thread
+ urlClassLoader = createClassLoader()
+ Thread.currentThread.setContextClassLoader(urlClassLoader)
+
// Initialize Spark environment (using system properties read above)
env = SparkEnv.createFromSystemProperties(slaveHostname, 0, false, false)
SparkEnv.set(env)
- // Create our ClassLoader (using spark properties) and set it on this thread
- classLoader = createClassLoader()
- Thread.currentThread.setContextClassLoader(classLoader)
-
// Start worker thread pool
threadPool = new ThreadPoolExecutor(
1, 128, 600, TimeUnit.SECONDS, new SynchronousQueue[Runnable])
@@ -54,15 +61,16 @@ class Executor extends Logging {
override def run() {
SparkEnv.set(env)
- Thread.currentThread.setContextClassLoader(classLoader)
+ Thread.currentThread.setContextClassLoader(urlClassLoader)
val ser = SparkEnv.get.closureSerializer.newInstance()
logInfo("Running task ID " + taskId)
context.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER)
try {
SparkEnv.set(env)
- Thread.currentThread.setContextClassLoader(classLoader)
Accumulators.clear()
- val task = ser.deserialize[Task[Any]](serializedTask, classLoader)
+ val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask)
+ updateDependencies(taskFiles, taskJars)
+ val task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader)
logInfo("Its generation is " + task.generation)
env.mapOutputTracker.updateGeneration(task.generation)
val value = task.run(taskId.toInt)
@@ -96,25 +104,15 @@ class Executor extends Logging {
* Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes
* created by the interpreter to the search path
*/
- private def createClassLoader(): ClassLoader = {
+ private def createClassLoader(): ExecutorURLClassLoader = {
var loader = this.getClass.getClassLoader
- // If any JAR URIs are given through spark.jar.uris, fetch them to the
- // current directory and put them all on the classpath. We assume that
- // each URL has a unique file name so that no local filenames will clash
- // in this process. This is guaranteed by ClusterScheduler.
- val uris = System.getProperty("spark.jar.uris", "")
- val localFiles = ArrayBuffer[String]()
- for (uri <- uris.split(",").filter(_.size > 0)) {
- val url = new URL(uri)
- val filename = url.getPath.split("/").last
- downloadFile(url, filename)
- localFiles += filename
- }
- if (localFiles.size > 0) {
- val urls = localFiles.map(f => new File(f).toURI.toURL).toArray
- loader = new URLClassLoader(urls, loader)
- }
+ // For each of the jars in the jarSet, add them to the class loader.
+ // We assume each of the files has already been fetched.
+ val urls = currentJars.keySet.map { uri =>
+ new File(uri.split("/").last).toURI.toURL
+ }.toArray
+ loader = new URLClassLoader(urls, loader)
// If the REPL is in use, add another ClassLoader that will read
// new classes defined by the REPL as the user types code
@@ -133,13 +131,31 @@ class Executor extends Logging {
}
}
- return loader
+ return new ExecutorURLClassLoader(Array(), loader)
}
- // Download a file from a given URL to the local filesystem
- private def downloadFile(url: URL, localPath: String) {
- val in = url.openStream()
- val out = new FileOutputStream(localPath)
- Utils.copyStream(in, out, true)
+ /**
+ * Download any missing dependencies if we receive a new set of files and JARs from the
+ * SparkContext. Also adds any new JARs we fetched to the class loader.
+ */
+ private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) {
+ // Fetch missing dependencies
+ for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
+ logInfo("Fetching " + name + " with timestamp " + timestamp)
+ Utils.fetchFile(name, new File("."))
+ currentFiles(name) = timestamp
+ }
+ for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
+ logInfo("Fetching " + name + " with timestamp " + timestamp)
+ Utils.fetchFile(name, new File("."))
+ currentJars(name) = timestamp
+ // Add it to our class loader
+ val localName = name.split("/").last
+ val url = new File(".", localName).toURI.toURL
+ if (!urlClassLoader.getURLs.contains(url)) {
+ logInfo("Adding " + url + " to class loader")
+ urlClassLoader.addURL(url)
+ }
+ }
}
}
diff --git a/core/src/main/scala/spark/executor/ExecutorBackend.scala b/core/src/main/scala/spark/executor/ExecutorBackend.scala
index 24c8776f31..e97e509700 100644
--- a/core/src/main/scala/spark/executor/ExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/ExecutorBackend.scala
@@ -6,6 +6,6 @@ import spark.TaskState.TaskState
/**
* A pluggable interface used by the Executor to send updates to the cluster scheduler.
*/
-trait ExecutorBackend {
+private[spark] trait ExecutorBackend {
def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer)
}
diff --git a/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
new file mode 100644
index 0000000000..5beb4d049e
--- /dev/null
+++ b/core/src/main/scala/spark/executor/ExecutorURLClassLoader.scala
@@ -0,0 +1,14 @@
+package spark.executor
+
+import java.net.{URLClassLoader, URL}
+
+/**
+ * The addURL method in URLClassLoader is protected. We subclass it to make this accessible.
+ */
+private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader)
+ extends URLClassLoader(urls, parent) {
+
+ override def addURL(url: URL) {
+ super.addURL(url)
+ }
+}
diff --git a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
index 50f4e41ede..eeab3959c6 100644
--- a/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/MesosExecutorBackend.scala
@@ -8,7 +8,7 @@ import com.google.protobuf.ByteString
import spark.{Utils, Logging}
import spark.TaskState
-class MesosExecutorBackend(executor: Executor)
+private[spark] class MesosExecutorBackend(executor: Executor)
extends MesosExecutor
with ExecutorBackend
with Logging {
@@ -59,7 +59,7 @@ class MesosExecutorBackend(executor: Executor)
/**
* Entry point for Mesos executor.
*/
-object MesosExecutorBackend {
+private[spark] object MesosExecutorBackend {
def main(args: Array[String]) {
MesosNativeLibrary.load()
// Create a new Executor and start it running
diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
index 26b163de0a..915f71ba9f 100644
--- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
@@ -14,7 +14,7 @@ import spark.scheduler.cluster.RegisterSlaveFailed
import spark.scheduler.cluster.RegisterSlave
-class StandaloneExecutorBackend(
+private[spark] class StandaloneExecutorBackend(
executor: Executor,
masterUrl: String,
slaveId: String,
@@ -62,7 +62,7 @@ class StandaloneExecutorBackend(
}
}
-object StandaloneExecutorBackend {
+private[spark] object StandaloneExecutorBackend {
def run(masterUrl: String, slaveId: String, hostname: String, cores: Int) {
// 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
diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala
index da8aff9dd5..80262ab7b4 100644
--- a/core/src/main/scala/spark/network/Connection.scala
+++ b/core/src/main/scala/spark/network/Connection.scala
@@ -11,6 +11,7 @@ import java.nio.channels.spi._
import java.net._
+private[spark]
abstract class Connection(val channel: SocketChannel, val selector: Selector) extends Logging {
channel.configureBlocking(false)
@@ -23,8 +24,8 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex
var onExceptionCallback: (Connection, Exception) => Unit = null
var onKeyInterestChangeCallback: (Connection, Int) => Unit = null
- lazy val remoteAddress = getRemoteAddress()
- lazy val remoteConnectionManagerId = ConnectionManagerId.fromSocketAddress(remoteAddress)
+ val remoteAddress = getRemoteAddress()
+ val remoteConnectionManagerId = ConnectionManagerId.fromSocketAddress(remoteAddress)
def key() = channel.keyFor(selector)
@@ -39,7 +40,10 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex
}
def close() {
- key.cancel()
+ val k = key()
+ if (k != null) {
+ k.cancel()
+ }
channel.close()
callOnCloseCallback()
}
@@ -99,7 +103,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector) ex
}
-class SendingConnection(val address: InetSocketAddress, selector_ : Selector)
+private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector)
extends Connection(SocketChannel.open, selector_) {
class Outbox(fair: Int = 0) {
@@ -134,9 +138,12 @@ extends Connection(SocketChannel.open, selector_) {
if (!message.started) logDebug("Starting to send [" + message + "]")
message.started = true
return chunk
+ } else {
+ /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/
+ message.finishTime = System.currentTimeMillis
+ logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId +
+ "] in " + message.timeTaken )
}
- /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/
- logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "] in " + message.timeTaken )
}
}
None
@@ -159,10 +166,11 @@ extends Connection(SocketChannel.open, selector_) {
}
logTrace("Sending chunk from [" + message+ "] to [" + remoteConnectionManagerId + "]")
return chunk
- }
- /*messages -= message*/
- message.finishTime = System.currentTimeMillis
- logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "] in " + message.timeTaken )
+ } else {
+ message.finishTime = System.currentTimeMillis
+ logDebug("Finished sending [" + message + "] to [" + remoteConnectionManagerId +
+ "] in " + message.timeTaken )
+ }
}
}
None
@@ -216,7 +224,7 @@ extends Connection(SocketChannel.open, selector_) {
while(true) {
if (currentBuffers.size == 0) {
outbox.synchronized {
- outbox.getChunk match {
+ outbox.getChunk() match {
case Some(chunk) => {
currentBuffers ++= chunk.buffers
}
@@ -252,7 +260,7 @@ extends Connection(SocketChannel.open, selector_) {
}
-class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector)
+private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector)
extends Connection(channel_, selector_) {
class Inbox() {
diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala
index 2bb5f5fc6b..da39108164 100644
--- a/core/src/main/scala/spark/network/ConnectionManager.scala
+++ b/core/src/main/scala/spark/network/ConnectionManager.scala
@@ -18,17 +18,17 @@ import akka.dispatch.{Await, Promise, ExecutionContext, Future}
import akka.util.Duration
import akka.util.duration._
-case class ConnectionManagerId(host: String, port: Int) {
+private[spark] case class ConnectionManagerId(host: String, port: Int) {
def toSocketAddress() = new InetSocketAddress(host, port)
}
-object ConnectionManagerId {
+private[spark] object ConnectionManagerId {
def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = {
new ConnectionManagerId(socketAddress.getHostName(), socketAddress.getPort())
}
}
-class ConnectionManager(port: Int) extends Logging {
+private[spark] class ConnectionManager(port: Int) extends Logging {
class MessageStatus(
val message: Message,
@@ -113,7 +113,7 @@ class ConnectionManager(port: Int) extends Logging {
val selectedKeysCount = selector.select()
if (selectedKeysCount == 0) {
- logInfo("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys")
+ logDebug("Selector selected " + selectedKeysCount + " of " + selector.keys.size + " keys")
}
if (selectorThread.isInterrupted) {
logInfo("Selector thread was interrupted!")
@@ -167,7 +167,6 @@ class ConnectionManager(port: Int) extends Logging {
}
def removeConnection(connection: Connection) {
- /*logInfo("Removing connection")*/
connectionsByKey -= connection.key
if (connection.isInstanceOf[SendingConnection]) {
val sendingConnection = connection.asInstanceOf[SendingConnection]
@@ -235,7 +234,7 @@ class ConnectionManager(port: Int) extends Logging {
def receiveMessage(connection: Connection, message: Message) {
val connectionManagerId = ConnectionManagerId.fromSocketAddress(message.senderAddress)
- logInfo("Received [" + message + "] from [" + connectionManagerId + "]")
+ logDebug("Received [" + message + "] from [" + connectionManagerId + "]")
val runnable = new Runnable() {
val creationTime = System.currentTimeMillis
def run() {
@@ -276,15 +275,15 @@ class ConnectionManager(port: Int) extends Logging {
logDebug("Calling back")
onReceiveCallback(bufferMessage, connectionManagerId)
} else {
- logWarning("Not calling back as callback is null")
+ logDebug("Not calling back as callback is null")
None
}
if (ackMessage.isDefined) {
if (!ackMessage.get.isInstanceOf[BufferMessage]) {
- logWarning("Response to " + bufferMessage + " is not a buffer message, it is of type " + ackMessage.get.getClass())
+ logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + ackMessage.get.getClass())
} else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) {
- logWarning("Response to " + bufferMessage + " does not have ack id set")
+ logDebug("Response to " + bufferMessage + " does not have ack id set")
ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id
}
}
@@ -349,7 +348,7 @@ class ConnectionManager(port: Int) extends Logging {
}
-object ConnectionManager {
+private[spark] object ConnectionManager {
def main(args: Array[String]) {
diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala
index 555b3454ee..47ceaf3c07 100644
--- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala
+++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala
@@ -11,7 +11,7 @@ import java.net.InetAddress
import akka.dispatch.Await
import akka.util.duration._
-object ConnectionManagerTest extends Logging{
+private[spark] object ConnectionManagerTest extends Logging{
def main(args: Array[String]) {
if (args.length < 2) {
println("Usage: ConnectionManagerTest <mesos cluster> <slaves file>")
diff --git a/core/src/main/scala/spark/network/Message.scala b/core/src/main/scala/spark/network/Message.scala
index 2e85803679..525751b5bf 100644
--- a/core/src/main/scala/spark/network/Message.scala
+++ b/core/src/main/scala/spark/network/Message.scala
@@ -7,8 +7,9 @@ import scala.collection.mutable.ArrayBuffer
import java.nio.ByteBuffer
import java.net.InetAddress
import java.net.InetSocketAddress
+import storage.BlockManager
-class MessageChunkHeader(
+private[spark] class MessageChunkHeader(
val typ: Long,
val id: Int,
val totalSize: Int,
@@ -36,7 +37,7 @@ class MessageChunkHeader(
" and sizes " + totalSize + " / " + chunkSize + " bytes"
}
-class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) {
+private[spark] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) {
val size = if (buffer == null) 0 else buffer.remaining
lazy val buffers = {
val ab = new ArrayBuffer[ByteBuffer]()
@@ -50,7 +51,7 @@ class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) {
override def toString = "" + this.getClass.getSimpleName + " (id = " + header.id + ", size = " + size + ")"
}
-abstract class Message(val typ: Long, val id: Int) {
+private[spark] abstract class Message(val typ: Long, val id: Int) {
var senderAddress: InetSocketAddress = null
var started = false
var startTime = -1L
@@ -64,10 +65,10 @@ abstract class Message(val typ: Long, val id: Int) {
def timeTaken(): String = (finishTime - startTime).toString + " ms"
- override def toString = "" + this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")"
+ override def toString = this.getClass.getSimpleName + "(id = " + id + ", size = " + size + ")"
}
-class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int)
+private[spark] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int)
extends Message(Message.BUFFER_MESSAGE, id_) {
val initialSize = currentSize()
@@ -97,10 +98,11 @@ extends Message(Message.BUFFER_MESSAGE, id_) {
while(!buffers.isEmpty) {
val buffer = buffers(0)
if (buffer.remaining == 0) {
+ BlockManager.dispose(buffer)
buffers -= buffer
} else {
val newBuffer = if (buffer.remaining <= maxChunkSize) {
- buffer.duplicate
+ buffer.duplicate()
} else {
buffer.slice().limit(maxChunkSize).asInstanceOf[ByteBuffer]
}
@@ -147,11 +149,10 @@ extends Message(Message.BUFFER_MESSAGE, id_) {
} else {
"BufferMessage(id = " + id + ", size = " + size + ")"
}
-
}
}
-object MessageChunkHeader {
+private[spark] object MessageChunkHeader {
val HEADER_SIZE = 40
def create(buffer: ByteBuffer): MessageChunkHeader = {
@@ -172,7 +173,7 @@ object MessageChunkHeader {
}
}
-object Message {
+private[spark] object Message {
val BUFFER_MESSAGE = 1111111111L
var lastId = 1
diff --git a/core/src/main/scala/spark/network/ReceiverTest.scala b/core/src/main/scala/spark/network/ReceiverTest.scala
index e1ba7c06c0..a174d5f403 100644
--- a/core/src/main/scala/spark/network/ReceiverTest.scala
+++ b/core/src/main/scala/spark/network/ReceiverTest.scala
@@ -3,7 +3,7 @@ package spark.network
import java.nio.ByteBuffer
import java.net.InetAddress
-object ReceiverTest {
+private[spark] object ReceiverTest {
def main(args: Array[String]) {
val manager = new ConnectionManager(9999)
diff --git a/core/src/main/scala/spark/network/SenderTest.scala b/core/src/main/scala/spark/network/SenderTest.scala
index 4ab6dd3414..a4ff69e4d2 100644
--- a/core/src/main/scala/spark/network/SenderTest.scala
+++ b/core/src/main/scala/spark/network/SenderTest.scala
@@ -3,7 +3,7 @@ package spark.network
import java.nio.ByteBuffer
import java.net.InetAddress
-object SenderTest {
+private[spark] object SenderTest {
def main(args: Array[String]) {
diff --git a/core/src/main/scala/spark/package.scala b/core/src/main/scala/spark/package.scala
new file mode 100644
index 0000000000..389ec4da3e
--- /dev/null
+++ b/core/src/main/scala/spark/package.scala
@@ -0,0 +1,15 @@
+/**
+ * Core Spark functionality. [[spark.SparkContext]] serves as the main entry point to Spark, while
+ * [[spark.RDD]] is the data type representing a distributed collection, and provides most
+ * parallel operations.
+ *
+ * In addition, [[spark.PairRDDFunctions]] contains operations available only on RDDs of key-value
+ * pairs, such as `groupByKey` and `join`; [[spark.DoubleRDDFunctions]] contains operations
+ * available only on RDDs of Doubles; and [[spark.SequenceFileRDDFunctions]] contains operations
+ * available on RDDs that can be saved as SequenceFiles. These operations are automatically
+ * available on any RDD of the right type (e.g. RDD[(Int, Int)] through implicit conversions when
+ * you `import spark.SparkContext._`.
+ */
+package object spark {
+ // For package docs only
+}
diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
index e6535836ab..42f46e06ed 100644
--- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala
+++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
@@ -12,7 +12,7 @@ import spark.scheduler.JobListener
* a result of type U for each partition, and that the action returns a partial or complete result
* of type R. Note that the type R must *include* any error bars on it (e.g. see BoundedInt).
*/
-class ApproximateActionListener[T, U, R](
+private[spark] class ApproximateActionListener[T, U, R](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
evaluator: ApproximateEvaluator[U, R],
diff --git a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
index 4772e43ef0..75713b2eaa 100644
--- a/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
+++ b/core/src/main/scala/spark/partial/ApproximateEvaluator.scala
@@ -4,7 +4,7 @@ package spark.partial
* An object that computes a function incrementally by merging in results of type U from multiple
* tasks. Allows partial evaluation at any point by calling currentResult().
*/
-trait ApproximateEvaluator[U, R] {
+private[spark] trait ApproximateEvaluator[U, R] {
def merge(outputId: Int, taskResult: U): Unit
def currentResult(): R
}
diff --git a/core/src/main/scala/spark/partial/CountEvaluator.scala b/core/src/main/scala/spark/partial/CountEvaluator.scala
index 1bc90d6b39..daf2c5170c 100644
--- a/core/src/main/scala/spark/partial/CountEvaluator.scala
+++ b/core/src/main/scala/spark/partial/CountEvaluator.scala
@@ -8,7 +8,7 @@ import cern.jet.stat.Probability
* TODO: There's currently a lot of shared code between this and GroupedCountEvaluator. It might
* be best to make this a special case of GroupedCountEvaluator with one group.
*/
-class CountEvaluator(totalOutputs: Int, confidence: Double)
+private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double)
extends ApproximateEvaluator[Long, BoundedDouble] {
var outputsMerged = 0
diff --git a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
index 3e631c0efc..01fbb8a11b 100644
--- a/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
+++ b/core/src/main/scala/spark/partial/GroupedCountEvaluator.scala
@@ -14,7 +14,7 @@ import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
/**
* An ApproximateEvaluator for counts by key. Returns a map of key to confidence interval.
*/
-class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double)
+private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double)
extends ApproximateEvaluator[OLMap[T], Map[T, BoundedDouble]] {
var outputsMerged = 0
diff --git a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
index 2a9ccba205..c622df5220 100644
--- a/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
+++ b/core/src/main/scala/spark/partial/GroupedMeanEvaluator.scala
@@ -12,7 +12,7 @@ import spark.util.StatCounter
/**
* An ApproximateEvaluator for means by key. Returns a map of key to confidence interval.
*/
-class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Double)
+private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Double)
extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] {
var outputsMerged = 0
diff --git a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
index 6a2ec7a7bd..20fa55cff2 100644
--- a/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
+++ b/core/src/main/scala/spark/partial/GroupedSumEvaluator.scala
@@ -12,7 +12,7 @@ import spark.util.StatCounter
/**
* An ApproximateEvaluator for sums by key. Returns a map of key to confidence interval.
*/
-class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Double)
+private[spark] class GroupedSumEvaluator[T](totalOutputs: Int, confidence: Double)
extends ApproximateEvaluator[JHashMap[T, StatCounter], Map[T, BoundedDouble]] {
var outputsMerged = 0
diff --git a/core/src/main/scala/spark/partial/MeanEvaluator.scala b/core/src/main/scala/spark/partial/MeanEvaluator.scala
index b8c7cb8863..762c85400d 100644
--- a/core/src/main/scala/spark/partial/MeanEvaluator.scala
+++ b/core/src/main/scala/spark/partial/MeanEvaluator.scala
@@ -7,7 +7,7 @@ import spark.util.StatCounter
/**
* An ApproximateEvaluator for means.
*/
-class MeanEvaluator(totalOutputs: Int, confidence: Double)
+private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double)
extends ApproximateEvaluator[StatCounter, BoundedDouble] {
var outputsMerged = 0
diff --git a/core/src/main/scala/spark/partial/StudentTCacher.scala b/core/src/main/scala/spark/partial/StudentTCacher.scala
index 6263ee3518..443abba5cd 100644
--- a/core/src/main/scala/spark/partial/StudentTCacher.scala
+++ b/core/src/main/scala/spark/partial/StudentTCacher.scala
@@ -7,7 +7,7 @@ import cern.jet.stat.Probability
* and various sample sizes. This is used by the MeanEvaluator to efficiently calculate
* confidence intervals for many keys.
*/
-class StudentTCacher(confidence: Double) {
+private[spark] class StudentTCacher(confidence: Double) {
val NORMAL_APPROX_SAMPLE_SIZE = 100 // For samples bigger than this, use Gaussian approximation
val normalApprox = Probability.normalInverse(1 - (1 - confidence) / 2)
val cache = Array.fill[Double](NORMAL_APPROX_SAMPLE_SIZE)(-1.0)
diff --git a/core/src/main/scala/spark/partial/SumEvaluator.scala b/core/src/main/scala/spark/partial/SumEvaluator.scala
index 0357a6bff8..58fb60f441 100644
--- a/core/src/main/scala/spark/partial/SumEvaluator.scala
+++ b/core/src/main/scala/spark/partial/SumEvaluator.scala
@@ -9,7 +9,7 @@ import spark.util.StatCounter
* together, then uses the formula for the variance of two independent random variables to get
* a variance for the result and compute a confidence interval.
*/
-class SumEvaluator(totalOutputs: Int, confidence: Double)
+private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double)
extends ApproximateEvaluator[StatCounter, BoundedDouble] {
var outputsMerged = 0
diff --git a/core/src/main/scala/spark/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala
index daabc0d566..cb73976aed 100644
--- a/core/src/main/scala/spark/BlockRDD.scala
+++ b/core/src/main/scala/spark/rdd/BlockRDD.scala
@@ -1,12 +1,18 @@
-package spark
+package spark.rdd
import scala.collection.mutable.HashMap
-class BlockRDDSplit(val blockId: String, idx: Int) extends Split {
+import spark.Dependency
+import spark.RDD
+import spark.SparkContext
+import spark.SparkEnv
+import spark.Split
+
+private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split {
val index = idx
}
-
+private[spark]
class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[String])
extends RDD[T](sc) {
diff --git a/core/src/main/scala/spark/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala
index e26041555a..7c354b6b2e 100644
--- a/core/src/main/scala/spark/CartesianRDD.scala
+++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala
@@ -1,9 +1,16 @@
-package spark
+package spark.rdd
+import spark.NarrowDependency
+import spark.RDD
+import spark.SparkContext
+import spark.Split
+
+private[spark]
class CartesianSplit(idx: Int, val s1: Split, val s2: Split) extends Split with Serializable {
override val index: Int = idx
}
+private[spark]
class CartesianRDD[T: ClassManifest, U:ClassManifest](
sc: SparkContext,
rdd1: RDD[T],
@@ -44,4 +51,4 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
def getParents(id: Int): Seq[Int] = List(id % numSplitsInRdd2)
}
)
-} \ No newline at end of file
+}
diff --git a/core/src/main/scala/spark/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
index 6959917d14..50bec9e63b 100644
--- a/core/src/main/scala/spark/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
@@ -1,21 +1,29 @@
-package spark
+package spark.rdd
-import java.net.URL
-import java.io.EOFException
-import java.io.ObjectInputStream
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.HashMap
-sealed trait CoGroupSplitDep extends Serializable
-case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep
-case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
+import spark.Aggregator
+import spark.Dependency
+import spark.Logging
+import spark.OneToOneDependency
+import spark.Partitioner
+import spark.RDD
+import spark.ShuffleDependency
+import spark.SparkEnv
+import spark.Split
+private[spark] sealed trait CoGroupSplitDep extends Serializable
+private[spark] case class NarrowCoGroupSplitDep(rdd: RDD[_], split: Split) extends CoGroupSplitDep
+private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
+
+private[spark]
class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Split with Serializable {
override val index: Int = idx
override def hashCode(): Int = idx
}
-class CoGroupAggregator
+private[spark] class CoGroupAggregator
extends Aggregator[Any, Any, ArrayBuffer[Any]](
{ x => ArrayBuffer(x) },
{ (b, x) => b += x },
@@ -31,13 +39,13 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
override val dependencies = {
val deps = new ArrayBuffer[Dependency[_]]
for ((rdd, index) <- rdds.zipWithIndex) {
- if (rdd.partitioner == Some(part)) {
- logInfo("Adding one-to-one dependency with " + rdd)
- deps += new OneToOneDependency(rdd)
+ val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true)
+ if (mapSideCombinedRDD.partitioner == Some(part)) {
+ logInfo("Adding one-to-one dependency with " + mapSideCombinedRDD)
+ deps += new OneToOneDependency(mapSideCombinedRDD)
} else {
logInfo("Adding shuffle dependency with " + rdd)
- deps += new ShuffleDependency[Any, Any, ArrayBuffer[Any]](
- context.newShuffleId, rdd, aggr, part)
+ deps += new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part)
}
}
deps.toList
@@ -50,7 +58,7 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
for (i <- 0 until array.size) {
array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (r, j) =>
dependencies(j) match {
- case s: ShuffleDependency[_, _, _] =>
+ case s: ShuffleDependency[_, _] =>
new ShuffleCoGroupSplitDep(s.shuffleId): CoGroupSplitDep
case _ =>
new NarrowCoGroupSplitDep(r, r.splits(i)): CoGroupSplitDep
@@ -82,13 +90,13 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner)
}
case ShuffleCoGroupSplitDep(shuffleId) => {
// Read map outputs of shuffle
- def mergePair(k: K, vs: Seq[Any]) {
- val mySeq = getSeq(k)
- for (v <- vs)
+ def mergePair(pair: (K, Seq[Any])) {
+ val mySeq = getSeq(pair._1)
+ for (v <- pair._2)
mySeq(depNum) += v
}
val fetcher = SparkEnv.get.shuffleFetcher
- fetcher.fetch[K, Seq[Any]](shuffleId, split.index, mergePair)
+ fetcher.fetch[K, Seq[Any]](shuffleId, split.index).foreach(mergePair)
}
}
map.iterator
diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
new file mode 100644
index 0000000000..0967f4f5df
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
@@ -0,0 +1,47 @@
+package spark.rdd
+
+import spark.NarrowDependency
+import spark.RDD
+import spark.Split
+
+private class CoalescedRDDSplit(val index: Int, val parents: Array[Split]) extends Split
+
+/**
+ * Coalesce the partitions of a parent RDD (`prev`) into fewer partitions, so that each partition of
+ * this RDD computes one or more of the parent ones. Will produce exactly `maxPartitions` if the
+ * parent had more than this many partitions, or fewer if the parent had fewer.
+ *
+ * This transformation is useful when an RDD with many partitions gets filtered into a smaller one,
+ * or to avoid having a large number of small tasks when processing a directory with many files.
+ */
+class CoalescedRDD[T: ClassManifest](prev: RDD[T], maxPartitions: Int)
+ extends RDD[T](prev.context) {
+
+ @transient val splits_ : Array[Split] = {
+ val prevSplits = prev.splits
+ if (prevSplits.length < maxPartitions) {
+ prevSplits.zipWithIndex.map{ case (s, idx) => new CoalescedRDDSplit(idx, Array(s)) }
+ } else {
+ (0 until maxPartitions).map { i =>
+ val rangeStart = (i * prevSplits.length) / maxPartitions
+ val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions
+ new CoalescedRDDSplit(i, prevSplits.slice(rangeStart, rangeEnd))
+ }.toArray
+ }
+ }
+
+ override def splits = splits_
+
+ override def compute(split: Split): Iterator[T] = {
+ split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap {
+ parentSplit => prev.iterator(parentSplit)
+ }
+ }
+
+ val dependencies = List(
+ new NarrowDependency(prev) {
+ def getParents(id: Int): Seq[Int] =
+ splits(id).asInstanceOf[CoalescedRDDSplit].parents.map(_.index)
+ }
+ )
+}
diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala
new file mode 100644
index 0000000000..dfe9dc73f3
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala
@@ -0,0 +1,12 @@
+package spark.rdd
+
+import spark.OneToOneDependency
+import spark.RDD
+import spark.Split
+
+private[spark]
+class FilteredRDD[T: ClassManifest](prev: RDD[T], f: T => Boolean) extends RDD[T](prev.context) {
+ override def splits = prev.splits
+ override val dependencies = List(new OneToOneDependency(prev))
+ override def compute(split: Split) = prev.iterator(split).filter(f)
+} \ No newline at end of file
diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
new file mode 100644
index 0000000000..3534dc8057
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
@@ -0,0 +1,16 @@
+package spark.rdd
+
+import spark.OneToOneDependency
+import spark.RDD
+import spark.Split
+
+private[spark]
+class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
+ prev: RDD[T],
+ f: T => TraversableOnce[U])
+ extends RDD[U](prev.context) {
+
+ override def splits = prev.splits
+ override val dependencies = List(new OneToOneDependency(prev))
+ override def compute(split: Split) = prev.iterator(split).flatMap(f)
+}
diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala
new file mode 100644
index 0000000000..e30564f2da
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala
@@ -0,0 +1,12 @@
+package spark.rdd
+
+import spark.OneToOneDependency
+import spark.RDD
+import spark.Split
+
+private[spark]
+class GlommedRDD[T: ClassManifest](prev: RDD[T]) extends RDD[Array[T]](prev.context) {
+ override def splits = prev.splits
+ override val dependencies = List(new OneToOneDependency(prev))
+ override def compute(split: Split) = Array(prev.iterator(split).toArray).iterator
+} \ No newline at end of file
diff --git a/core/src/main/scala/spark/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala
index 0befca582d..bf29a1f075 100644
--- a/core/src/main/scala/spark/HadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
import java.io.EOFException
import java.util.NoSuchElementException
@@ -15,10 +15,16 @@ import org.apache.hadoop.mapred.RecordReader
import org.apache.hadoop.mapred.Reporter
import org.apache.hadoop.util.ReflectionUtils
+import spark.Dependency
+import spark.RDD
+import spark.SerializableWritable
+import spark.SparkContext
+import spark.Split
+
/**
* A Spark split class that wraps around a Hadoop InputSplit.
*/
-class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit)
+private[spark] class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit)
extends Split
with Serializable {
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
new file mode 100644
index 0000000000..a904ef62c3
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
@@ -0,0 +1,19 @@
+package spark.rdd
+
+import spark.OneToOneDependency
+import spark.RDD
+import spark.Split
+
+private[spark]
+class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
+ prev: RDD[T],
+ f: Iterator[T] => Iterator[U],
+ preservesPartitioning: Boolean = false)
+ extends RDD[U](prev.context) {
+
+ override val partitioner = if (preservesPartitioning) prev.partitioner else None
+
+ override def splits = prev.splits
+ override val dependencies = List(new OneToOneDependency(prev))
+ override def compute(split: Split) = f(prev.iterator(split))
+} \ No newline at end of file
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
new file mode 100644
index 0000000000..adc541694e
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
@@ -0,0 +1,21 @@
+package spark.rdd
+
+import spark.OneToOneDependency
+import spark.RDD
+import spark.Split
+
+/**
+ * A variant of the MapPartitionsRDD that passes the split index into the
+ * closure. This can be used to generate or collect partition specific
+ * information such as the number of tuples in a partition.
+ */
+private[spark]
+class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest](
+ prev: RDD[T],
+ f: (Int, Iterator[T]) => Iterator[U])
+ extends RDD[U](prev.context) {
+
+ override def splits = prev.splits
+ override val dependencies = List(new OneToOneDependency(prev))
+ override def compute(split: Split) = f(split.index, prev.iterator(split))
+} \ No newline at end of file
diff --git a/core/src/main/scala/spark/rdd/MappedRDD.scala b/core/src/main/scala/spark/rdd/MappedRDD.scala
new file mode 100644
index 0000000000..59bedad8ef
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/MappedRDD.scala
@@ -0,0 +1,16 @@
+package spark.rdd
+
+import spark.OneToOneDependency
+import spark.RDD
+import spark.Split
+
+private[spark]
+class MappedRDD[U: ClassManifest, T: ClassManifest](
+ prev: RDD[T],
+ f: T => U)
+ extends RDD[U](prev.context) {
+
+ override def splits = prev.splits
+ override val dependencies = List(new OneToOneDependency(prev))
+ override def compute(split: Split) = prev.iterator(split).map(f)
+} \ No newline at end of file
diff --git a/core/src/main/scala/spark/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
index 14f708a3f8..7a1a0fb87d 100644
--- a/core/src/main/scala/spark/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
@@ -1,18 +1,19 @@
-package spark
+package spark.rdd
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io.Writable
-import org.apache.hadoop.mapreduce.InputFormat
-import org.apache.hadoop.mapreduce.InputSplit
-import org.apache.hadoop.mapreduce.JobContext
-import org.apache.hadoop.mapreduce.JobID
-import org.apache.hadoop.mapreduce.RecordReader
-import org.apache.hadoop.mapreduce.TaskAttemptContext
-import org.apache.hadoop.mapreduce.TaskAttemptID
+import org.apache.hadoop.mapreduce._
import java.util.Date
import java.text.SimpleDateFormat
+import spark.Dependency
+import spark.RDD
+import spark.SerializableWritable
+import spark.SparkContext
+import spark.Split
+
+private[spark]
class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable)
extends Split {
@@ -26,7 +27,8 @@ class NewHadoopRDD[K, V](
inputFormatClass: Class[_ <: InputFormat[K, V]],
keyClass: Class[K], valueClass: Class[V],
@transient conf: Configuration)
- extends RDD[(K, V)](sc) {
+ extends RDD[(K, V)](sc)
+ with HadoopMapReduceUtil {
// A Hadoop Configuration can be about 10 KB, which is pretty big, so broadcast it
val confBroadcast = sc.broadcast(new SerializableWritable(conf))
@@ -43,7 +45,7 @@ class NewHadoopRDD[K, V](
@transient
private val splits_ : Array[Split] = {
val inputFormat = inputFormatClass.newInstance
- val jobContext = new JobContext(conf, jobId)
+ val jobContext = newJobContext(conf, jobId)
val rawSplits = inputFormat.getSplits(jobContext).toArray
val result = new Array[Split](rawSplits.size)
for (i <- 0 until rawSplits.size) {
@@ -58,7 +60,7 @@ class NewHadoopRDD[K, V](
val split = theSplit.asInstanceOf[NewHadoopSplit]
val conf = confBroadcast.value.value
val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0)
- val context = new TaskAttemptContext(conf, attemptId)
+ val context = newTaskAttemptContext(conf, attemptId)
val format = inputFormatClass.newInstance
val reader = format.createRecordReader(split.serializableHadoopSplit.value, context)
reader.initialize(split.serializableHadoopSplit.value, context)
diff --git a/core/src/main/scala/spark/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala
index 3103d7889b..98ea0c92d6 100644
--- a/core/src/main/scala/spark/PipedRDD.scala
+++ b/core/src/main/scala/spark/rdd/PipedRDD.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
import java.io.PrintWriter
import java.util.StringTokenizer
@@ -8,6 +8,12 @@ import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
+import spark.OneToOneDependency
+import spark.RDD
+import spark.SparkEnv
+import spark.Split
+
+
/**
* 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.
diff --git a/core/src/main/scala/spark/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala
index 8ef40d8d9e..87a5268f27 100644
--- a/core/src/main/scala/spark/SampledRDD.scala
+++ b/core/src/main/scala/spark/rdd/SampledRDD.scala
@@ -1,7 +1,14 @@
-package spark
+package spark.rdd
import java.util.Random
+import cern.jet.random.Poisson
+import cern.jet.random.engine.DRand
+import spark.RDD
+import spark.OneToOneDependency
+import spark.Split
+
+private[spark]
class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable {
override val index: Int = prev.index
}
@@ -28,19 +35,21 @@ class SampledRDD[T: ClassManifest](
override def compute(splitIn: Split) = {
val split = splitIn.asInstanceOf[SampledRDDSplit]
- val rg = new Random(split.seed)
- // Sampling with replacement (TODO: use reservoir sampling to make this more efficient?)
if (withReplacement) {
- val oldData = prev.iterator(split.prev).toArray
- val sampleSize = (oldData.size * frac).ceil.toInt
- val sampledData = {
- // all of oldData's indices are candidates, even if sampleSize < oldData.size
- for (i <- 1 to sampleSize)
- yield oldData(rg.nextInt(oldData.size))
+ // For large datasets, the expected number of occurrences of each element in a sample with
+ // replacement is Poisson(frac). We use that to get a count for each element.
+ val poisson = new Poisson(frac, new DRand(split.seed))
+ prev.iterator(split.prev).flatMap { element =>
+ val count = poisson.nextInt()
+ if (count == 0) {
+ Iterator.empty // Avoid object allocation when we return 0 items, which is quite often
+ } else {
+ Iterator.fill(count)(element)
+ }
}
- sampledData.iterator
} else { // Sampling without replacement
- prev.iterator(split.prev).filter(x => (rg.nextDouble <= frac))
+ val rand = new Random(split.seed)
+ prev.iterator(split.prev).filter(x => (rand.nextDouble <= frac))
}
}
}
diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
new file mode 100644
index 0000000000..145e419c53
--- /dev/null
+++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
@@ -0,0 +1,40 @@
+package spark.rdd
+
+import spark.Partitioner
+import spark.RDD
+import spark.ShuffleDependency
+import spark.SparkEnv
+import spark.Split
+
+private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
+ override val index = idx
+ override def hashCode(): Int = idx
+}
+
+/**
+ * The resulting RDD from a shuffle (e.g. repartitioning of data).
+ * @param parent the parent RDD.
+ * @param part the partitioner used to partition the RDD
+ * @tparam K the key class.
+ * @tparam V the value class.
+ */
+class ShuffledRDD[K, V](
+ @transient parent: RDD[(K, V)],
+ part: Partitioner) extends RDD[(K, V)](parent.context) {
+
+ override val partitioner = Some(part)
+
+ @transient
+ val splits_ = Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
+
+ override def splits = splits_
+
+ override def preferredLocations(split: Split) = Nil
+
+ val dep = new ShuffleDependency(parent, part)
+ override val dependencies = List(dep)
+
+ override def compute(split: Split): Iterator[(K, V)] = {
+ SparkEnv.get.shuffleFetcher.fetch[K, V](dep.shuffleId, split.index)
+ }
+}
diff --git a/core/src/main/scala/spark/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala
index 0e8164d6ab..f0b9225f7c 100644
--- a/core/src/main/scala/spark/UnionRDD.scala
+++ b/core/src/main/scala/spark/rdd/UnionRDD.scala
@@ -1,8 +1,14 @@
-package spark
+package spark.rdd
import scala.collection.mutable.ArrayBuffer
-class UnionSplit[T: ClassManifest](
+import spark.Dependency
+import spark.RangeDependency
+import spark.RDD
+import spark.SparkContext
+import spark.Split
+
+private[spark] class UnionSplit[T: ClassManifest](
idx: Int,
rdd: RDD[T],
split: Split)
@@ -37,7 +43,7 @@ class UnionRDD[T: ClassManifest](
override val dependencies = {
val deps = new ArrayBuffer[Dependency[_]]
var pos = 0
- for ((rdd, index) <- rdds.zipWithIndex) {
+ for (rdd <- rdds) {
deps += new RangeDependency(rdd, 0, pos, rdd.splits.size)
pos += rdd.splits.size
}
diff --git a/core/src/main/scala/spark/scheduler/ActiveJob.scala b/core/src/main/scala/spark/scheduler/ActiveJob.scala
index 0ecff9ce77..5a4e9a582d 100644
--- a/core/src/main/scala/spark/scheduler/ActiveJob.scala
+++ b/core/src/main/scala/spark/scheduler/ActiveJob.scala
@@ -5,11 +5,12 @@ import spark.TaskContext
/**
* Tracks information about an active job in the DAGScheduler.
*/
-class ActiveJob(
+private[spark] class ActiveJob(
val runId: Int,
val finalStage: Stage,
val func: (TaskContext, Iterator[_]) => _,
val partitions: Array[Int],
+ val callSite: String,
val listener: JobListener) {
val numPartitions = partitions.length
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index f7472971b5..aaaed59c4a 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -21,6 +21,7 @@ import spark.storage.BlockManagerId
* schedule to run the job. Subclasses only need to implement the code to send a task to the cluster
* and to report fetch failures (the submitTasks method, and code to add CompletionEvents).
*/
+private[spark]
class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with Logging {
taskSched.setListener(this)
@@ -38,6 +39,11 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
eventQueue.put(HostLost(host))
}
+ // Called by TaskScheduler to cancel an entier TaskSet due to repeated failures.
+ override def taskSetFailed(taskSet: TaskSet, reason: String) {
+ eventQueue.put(TaskSetFailed(taskSet, reason))
+ }
+
// 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
@@ -98,7 +104,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
* The priority value passed in will be used if the stage doesn't already exist with
* a lower priority (we assume that priorities always increase across jobs for now).
*/
- def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_,_], priority: Int): Stage = {
+ def getShuffleMapStage(shuffleDep: ShuffleDependency[_,_], priority: Int): Stage = {
shuffleToMapStage.get(shuffleDep.shuffleId) match {
case Some(stage) => stage
case None =>
@@ -113,10 +119,10 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
* as a result stage for the final RDD used directly in an action. The stage will also be given
* the provided priority.
*/
- def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_,_]], priority: Int): Stage = {
+ def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = {
// 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 splits is unknown
- logInfo("Registering RDD " + rdd.id + ": " + rdd)
+ logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")")
cacheTracker.registerRDD(rdd.id, rdd.splits.size)
if (shuffleDep != None) {
mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.splits.size)
@@ -139,11 +145,11 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
visited += r
// Kind of ugly: need to register RDDs with the cache here since
// we can't do it in its constructor because # of splits is unknown
- logInfo("Registering parent RDD " + r.id + ": " + r)
+ logInfo("Registering parent RDD " + r.id + " (" + r.origin + ")")
cacheTracker.registerRDD(r.id, r.splits.size)
for (dep <- r.dependencies) {
dep match {
- case shufDep: ShuffleDependency[_,_,_] =>
+ case shufDep: ShuffleDependency[_,_] =>
parents += getShuffleMapStage(shufDep, priority)
case _ =>
visit(dep.rdd)
@@ -166,7 +172,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
if (locs(p) == Nil) {
for (dep <- rdd.dependencies) {
dep match {
- case shufDep: ShuffleDependency[_,_,_] =>
+ case shufDep: ShuffleDependency[_,_] =>
val mapStage = getShuffleMapStage(shufDep, stage.priority)
if (!mapStage.isAvailable) {
missing += mapStage
@@ -183,23 +189,25 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
missing.toList
}
- def runJob[T, U](
+ def runJob[T, U: ClassManifest](
finalRdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
+ callSite: String,
allowLocal: Boolean)
- (implicit m: ClassManifest[U]): Array[U] =
+ : Array[U] =
{
if (partitions.size == 0) {
return new Array[U](0)
}
val waiter = new JobWaiter(partitions.size)
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
- eventQueue.put(JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, waiter))
+ eventQueue.put(JobSubmitted(finalRdd, func2, partitions.toArray, allowLocal, callSite, waiter))
waiter.getResult() match {
case JobSucceeded(results: Seq[_]) =>
return results.asInstanceOf[Seq[U]].toArray
case JobFailed(exception: Exception) =>
+ logInfo("Failed to run " + callSite)
throw exception
}
}
@@ -208,13 +216,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
evaluator: ApproximateEvaluator[U, R],
- timeout: Long
- ): PartialResult[R] =
+ callSite: String,
+ timeout: Long)
+ : PartialResult[R] =
{
val listener = new ApproximateActionListener(rdd, func, evaluator, timeout)
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
val partitions = (0 until rdd.splits.size).toArray
- eventQueue.put(JobSubmitted(rdd, func2, partitions, false, listener))
+ eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener))
return listener.getResult() // Will throw an exception if the job fails
}
@@ -234,13 +243,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
}
event match {
- case JobSubmitted(finalRDD, func, partitions, allowLocal, listener) =>
+ case JobSubmitted(finalRDD, func, partitions, allowLocal, callSite, listener) =>
val runId = nextRunId.getAndIncrement()
val finalStage = newStage(finalRDD, None, runId)
- val job = new ActiveJob(runId, finalStage, func, partitions, listener)
+ val job = new ActiveJob(runId, finalStage, func, partitions, callSite, listener)
updateCacheLocs()
- logInfo("Got job " + job.runId + " with " + partitions.length + " output partitions")
- logInfo("Final stage: " + finalStage)
+ logInfo("Got job " + job.runId + " (" + callSite + ") with " + partitions.length +
+ " output partitions")
+ logInfo("Final stage: " + finalStage + " (" + finalStage.origin + ")")
logInfo("Parents of final stage: " + finalStage.parents)
logInfo("Missing parents: " + getMissingParentStages(finalStage))
if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) {
@@ -258,6 +268,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
case completion: CompletionEvent =>
handleTaskCompletion(completion)
+ case TaskSetFailed(taskSet, reason) =>
+ abortStage(idToStage(taskSet.stageId), reason)
+
case StopDAGScheduler =>
// Cancel any active jobs
for (job <- activeJobs) {
@@ -329,7 +342,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
val missing = getMissingParentStages(stage).sortBy(_.id)
logDebug("missing: " + missing)
if (missing == Nil) {
- logInfo("Submitting " + stage + ", which has no missing parents")
+ logInfo("Submitting " + stage + " (" + stage.origin + "), which has no missing parents")
submitMissingTasks(stage)
running += stage
} else {
@@ -409,14 +422,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
case smt: ShuffleMapTask =>
val stage = idToStage(smt.stageId)
- val bmAddress = event.result.asInstanceOf[BlockManagerId]
- val host = bmAddress.ip
+ val status = event.result.asInstanceOf[MapStatus]
+ val host = status.address.ip
logInfo("ShuffleMapTask finished with host " + host)
if (!deadHosts.contains(host)) { // TODO: Make sure hostnames are consistent with Mesos
- stage.addOutputLoc(smt.partition, bmAddress)
+ stage.addOutputLoc(smt.partition, status)
}
if (running.contains(stage) && pendingTasks(stage).isEmpty) {
- logInfo(stage + " finished; looking for newly runnable stages")
+ logInfo(stage + " (" + stage.origin + ") finished; looking for newly runnable stages")
running -= stage
logInfo("running: " + running)
logInfo("waiting: " + waiting)
@@ -430,7 +443,8 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
if (stage.outputLocs.count(_ == Nil) != 0) {
// Some tasks had failed; let's resubmit this stage
// TODO: Lower-level scheduler should also deal with this
- logInfo("Resubmitting " + stage + " because some of its tasks had failed: " +
+ logInfo("Resubmitting " + stage + " (" + stage.origin +
+ ") because some of its tasks had failed: " +
stage.outputLocs.zipWithIndex.filter(_._1 == Nil).map(_._2).mkString(", "))
submitStage(stage)
} else {
@@ -444,6 +458,7 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
waiting --= newlyRunnable
running ++= newlyRunnable
for (stage <- newlyRunnable.sortBy(_.id)) {
+ logInfo("Submitting " + stage + " (" + stage.origin + "), which is now runnable")
submitMissingTasks(stage)
}
}
@@ -460,12 +475,14 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
running -= failedStage
failed += failedStage
// TODO: Cancel running tasks in the stage
- logInfo("Marking " + failedStage + " for resubmision due to a fetch failure")
+ logInfo("Marking " + failedStage + " (" + failedStage.origin +
+ ") for resubmision due to a fetch failure")
// Mark the map whose fetch failed as broken in the map stage
val mapStage = shuffleToMapStage(shuffleId)
mapStage.removeOutputLoc(mapId, bmAddress)
mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress)
- logInfo("The failed fetch was from " + mapStage + "; marking it for resubmission")
+ logInfo("The failed fetch was from " + mapStage + " (" + mapStage.origin +
+ "); marking it for resubmission")
failed += mapStage
// Remember that a fetch failed now; this is used to resubmit the broken
// stages later, after a small wait (to give other tasks the chance to fail)
@@ -475,18 +492,9 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
handleHostLost(bmAddress.ip)
}
- case _ =>
- // Non-fetch failure -- probably a bug in the job, so bail out
- // TODO: Cancel all tasks that are still running
- resultStageToJob.get(stage) match {
- case Some(job) =>
- val error = new SparkException("Task failed: " + task + ", reason: " + event.reason)
- job.listener.jobFailed(error)
- activeJobs -= job
- resultStageToJob -= stage
- case None =>
- logInfo("Ignoring result from " + task + " because its job has finished")
- }
+ case other =>
+ // Non-fetch failure -- probably a bug in user code; abort all jobs depending on this stage
+ abortStage(idToStage(task.stageId), task + " failed: " + other)
}
}
@@ -509,6 +517,53 @@ class DAGScheduler(taskSched: TaskScheduler) extends TaskSchedulerListener with
updateCacheLocs()
}
}
+
+ /**
+ * Aborts all jobs depending on a particular Stage. This is called in response to a task set
+ * being cancelled by the TaskScheduler. Use taskSetFailed() to inject this event from outside.
+ */
+ def abortStage(failedStage: Stage, reason: String) {
+ val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq
+ for (resultStage <- dependentStages) {
+ val job = resultStageToJob(resultStage)
+ job.listener.jobFailed(new SparkException("Job failed: " + reason))
+ activeJobs -= job
+ resultStageToJob -= resultStage
+ }
+ if (dependentStages.isEmpty) {
+ logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done")
+ }
+ }
+
+ /**
+ * Return true if one of stage's ancestors is target.
+ */
+ def stageDependsOn(stage: Stage, target: Stage): Boolean = {
+ if (stage == target) {
+ return true
+ }
+ val visitedRdds = new HashSet[RDD[_]]
+ val visitedStages = new HashSet[Stage]
+ def visit(rdd: RDD[_]) {
+ if (!visitedRdds(rdd)) {
+ visitedRdds += rdd
+ for (dep <- rdd.dependencies) {
+ dep match {
+ case shufDep: ShuffleDependency[_,_] =>
+ val mapStage = getShuffleMapStage(shufDep, stage.priority)
+ if (!mapStage.isAvailable) {
+ visitedStages += mapStage
+ visit(mapStage.rdd)
+ } // Otherwise there's no need to follow the dependency back
+ case narrowDep: NarrowDependency[_] =>
+ visit(narrowDep.rdd)
+ }
+ }
+ }
+ }
+ visit(stage.rdd)
+ visitedRdds.contains(target.rdd)
+ }
def getPreferredLocs(rdd: RDD[_], partition: Int): List[String] = {
// If the partition is cached, return the cache locations
diff --git a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
index 0fc73059c3..3422a21d9d 100644
--- a/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/spark/scheduler/DAGSchedulerEvent.scala
@@ -10,23 +10,26 @@ import spark._
* submitted) but there is a single "logic" thread that reads these events and takes decisions.
* This greatly simplifies synchronization.
*/
-sealed trait DAGSchedulerEvent
+private[spark] sealed trait DAGSchedulerEvent
-case class JobSubmitted(
+private[spark] case class JobSubmitted(
finalRDD: RDD[_],
func: (TaskContext, Iterator[_]) => _,
partitions: Array[Int],
allowLocal: Boolean,
+ callSite: String,
listener: JobListener)
extends DAGSchedulerEvent
-case class CompletionEvent(
+private[spark] case class CompletionEvent(
task: Task[_],
reason: TaskEndReason,
result: Any,
accumUpdates: Map[Long, Any])
extends DAGSchedulerEvent
-case class HostLost(host: String) extends DAGSchedulerEvent
+private[spark] case class HostLost(host: String) extends DAGSchedulerEvent
-case object StopDAGScheduler extends DAGSchedulerEvent
+private[spark] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent
+
+private[spark] case object StopDAGScheduler extends DAGSchedulerEvent
diff --git a/core/src/main/scala/spark/scheduler/JobListener.scala b/core/src/main/scala/spark/scheduler/JobListener.scala
index d4dd536a7d..f46b9d551d 100644
--- a/core/src/main/scala/spark/scheduler/JobListener.scala
+++ b/core/src/main/scala/spark/scheduler/JobListener.scala
@@ -5,7 +5,7 @@ package spark.scheduler
* DAGScheduler. The listener is notified each time a task succeeds, as well as if the whole
* job fails (and no further taskSucceeded events will happen).
*/
-trait JobListener {
+private[spark] trait JobListener {
def taskSucceeded(index: Int, result: Any)
def jobFailed(exception: Exception)
}
diff --git a/core/src/main/scala/spark/scheduler/JobResult.scala b/core/src/main/scala/spark/scheduler/JobResult.scala
index 62b458eccb..c4a74e526f 100644
--- a/core/src/main/scala/spark/scheduler/JobResult.scala
+++ b/core/src/main/scala/spark/scheduler/JobResult.scala
@@ -3,7 +3,7 @@ package spark.scheduler
/**
* A result of a job in the DAGScheduler.
*/
-sealed trait JobResult
+private[spark] sealed trait JobResult
-case class JobSucceeded(results: Seq[_]) extends JobResult
-case class JobFailed(exception: Exception) extends JobResult
+private[spark] case class JobSucceeded(results: Seq[_]) extends JobResult
+private[spark] case class JobFailed(exception: Exception) extends JobResult
diff --git a/core/src/main/scala/spark/scheduler/JobWaiter.scala b/core/src/main/scala/spark/scheduler/JobWaiter.scala
index 4c2ae23051..b3d4feebe5 100644
--- a/core/src/main/scala/spark/scheduler/JobWaiter.scala
+++ b/core/src/main/scala/spark/scheduler/JobWaiter.scala
@@ -5,7 +5,7 @@ import scala.collection.mutable.ArrayBuffer
/**
* An object that waits for a DAGScheduler job to complete.
*/
-class JobWaiter(totalTasks: Int) extends JobListener {
+private[spark] class JobWaiter(totalTasks: Int) extends JobListener {
private val taskResults = ArrayBuffer.fill[Any](totalTasks)(null)
private var finishedTasks = 0
diff --git a/core/src/main/scala/spark/scheduler/MapStatus.scala b/core/src/main/scala/spark/scheduler/MapStatus.scala
new file mode 100644
index 0000000000..4532d9497f
--- /dev/null
+++ b/core/src/main/scala/spark/scheduler/MapStatus.scala
@@ -0,0 +1,27 @@
+package spark.scheduler
+
+import spark.storage.BlockManagerId
+import java.io.{ObjectOutput, ObjectInput, Externalizable}
+
+/**
+ * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the
+ * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks.
+ * The map output sizes are compressed using MapOutputTracker.compressSize.
+ */
+private[spark] class MapStatus(var address: BlockManagerId, var compressedSizes: Array[Byte])
+ extends Externalizable {
+
+ def this() = this(null, null) // For deserialization only
+
+ def writeExternal(out: ObjectOutput) {
+ address.writeExternal(out)
+ out.writeInt(compressedSizes.length)
+ out.write(compressedSizes)
+ }
+
+ def readExternal(in: ObjectInput) {
+ address = new BlockManagerId(in)
+ compressedSizes = new Array[Byte](in.readInt())
+ in.readFully(compressedSizes)
+ }
+}
diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala
index 090ced9d76..2ebd4075a2 100644
--- a/core/src/main/scala/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/spark/scheduler/ResultTask.scala
@@ -2,7 +2,7 @@ package spark.scheduler
import spark._
-class ResultTask[T, U](
+private[spark] class ResultTask[T, U](
stageId: Int,
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
index f1eae9bc88..60105c42b6 100644
--- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
@@ -1,10 +1,10 @@
package spark.scheduler
import java.io._
-import java.util.HashMap
+import java.util.{HashMap => JHashMap}
import java.util.zip.{GZIPInputStream, GZIPOutputStream}
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.{ArrayBuffer, HashMap}
import scala.collection.JavaConversions._
import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
@@ -15,11 +15,14 @@ import com.ning.compress.lzf.LZFOutputStream
import spark._
import spark.storage._
-object ShuffleMapTask {
- val serializedInfoCache = new HashMap[Int, Array[Byte]]
- val deserializedInfoCache = new HashMap[Int, (RDD[_], ShuffleDependency[_,_,_])]
+private[spark] object ShuffleMapTask {
- def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_,_]): Array[Byte] = {
+ // A simple map between the stage id to the serialized byte array of a task.
+ // Served as a cache for task serialization because serialization can be
+ // expensive on the master node if it needs to launch thousands of tasks.
+ val serializedInfoCache = new JHashMap[Int, Array[Byte]]
+
+ def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = {
synchronized {
val old = serializedInfoCache.get(stageId)
if (old != null) {
@@ -38,40 +41,40 @@ object ShuffleMapTask {
}
}
- def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_,_,_]) = {
+ def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_,_]) = {
synchronized {
- val old = deserializedInfoCache.get(stageId)
- if (old != null) {
- return old
- } else {
- val loader = Thread.currentThread.getContextClassLoader
- val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
- val ser = SparkEnv.get.closureSerializer.newInstance
- val objIn = ser.deserializeStream(in)
- val rdd = objIn.readObject().asInstanceOf[RDD[_]]
- val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_,_]]
- val tuple = (rdd, dep)
- deserializedInfoCache.put(stageId, tuple)
- return tuple
- }
+ val loader = Thread.currentThread.getContextClassLoader
+ val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
+ val ser = SparkEnv.get.closureSerializer.newInstance
+ val objIn = ser.deserializeStream(in)
+ val rdd = objIn.readObject().asInstanceOf[RDD[_]]
+ val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_,_]]
+ return (rdd, dep)
}
}
+ // Since both the JarSet and FileSet have the same format this is used for both.
+ def deserializeFileSet(bytes: Array[Byte]) : HashMap[String, Long] = {
+ val in = new GZIPInputStream(new ByteArrayInputStream(bytes))
+ val objIn = new ObjectInputStream(in)
+ val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap
+ return (HashMap(set.toSeq: _*))
+ }
+
def clearCache() {
synchronized {
serializedInfoCache.clear()
- deserializedInfoCache.clear()
}
}
}
-class ShuffleMapTask(
+private[spark] class ShuffleMapTask(
stageId: Int,
var rdd: RDD[_],
- var dep: ShuffleDependency[_,_,_],
+ var dep: ShuffleDependency[_,_],
var partition: Int,
@transient var locs: Seq[String])
- extends Task[BlockManagerId](stageId)
+ extends Task[MapStatus](stageId)
with Externalizable
with Logging {
@@ -106,32 +109,31 @@ class ShuffleMapTask(
split = in.readObject().asInstanceOf[Split]
}
- override def run(attemptId: Long): BlockManagerId = {
+ override def run(attemptId: Long): MapStatus = {
val numOutputSplits = dep.partitioner.numPartitions
- val aggregator = dep.aggregator.asInstanceOf[Aggregator[Any, Any, Any]]
val partitioner = dep.partitioner
- val buckets = Array.tabulate(numOutputSplits)(_ => new HashMap[Any, Any])
+
+ // Partition the map output.
+ val buckets = Array.fill(numOutputSplits)(new ArrayBuffer[(Any, Any)])
for (elem <- rdd.iterator(split)) {
- val (k, v) = elem.asInstanceOf[(Any, Any)]
- var bucketId = partitioner.getPartition(k)
- val bucket = buckets(bucketId)
- var existing = bucket.get(k)
- if (existing == null) {
- bucket.put(k, aggregator.createCombiner(v))
- } else {
- bucket.put(k, aggregator.mergeValue(existing, v))
- }
+ val pair = elem.asInstanceOf[(Any, Any)]
+ val bucketId = partitioner.getPartition(pair._1)
+ buckets(bucketId) += pair
}
- val ser = SparkEnv.get.serializer.newInstance()
+ val bucketIterators = buckets.map(_.iterator)
+
+ val compressedSizes = new Array[Byte](numOutputSplits)
+
val blockManager = SparkEnv.get.blockManager
for (i <- 0 until numOutputSplits) {
- val blockId = "shuffleid_" + dep.shuffleId + "_" + partition + "_" + i
- // Get a scala iterator from java map
- val iter: Iterator[(Any, Any)] = buckets(i).iterator
- // TODO: This should probably be DISK_ONLY
- blockManager.put(blockId, iter, StorageLevel.MEMORY_ONLY, false)
+ val blockId = "shuffle_" + dep.shuffleId + "_" + partition + "_" + i
+ // Get a Scala iterator from Java map
+ val iter: Iterator[(Any, Any)] = bucketIterators(i)
+ val size = blockManager.put(blockId, iter, StorageLevel.DISK_ONLY, false)
+ compressedSizes(i) = MapOutputTracker.compressSize(size)
}
- return SparkEnv.get.blockManager.blockManagerId
+
+ return new MapStatus(blockManager.blockManagerId, compressedSizes)
}
override def preferredLocations: Seq[String] = locs
diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala
index cd660c9085..4846b66729 100644
--- a/core/src/main/scala/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/spark/scheduler/Stage.scala
@@ -19,39 +19,39 @@ import spark.storage.BlockManagerId
* Each Stage also has a priority, which is (by default) based on the job it was submitted in.
* This allows Stages from earlier jobs to be computed first or recovered faster on failure.
*/
-class Stage(
+private[spark] class Stage(
val id: Int,
val rdd: RDD[_],
- val shuffleDep: Option[ShuffleDependency[_,_,_]], // Output shuffle if stage is a map stage
+ val shuffleDep: Option[ShuffleDependency[_,_]], // Output shuffle if stage is a map stage
val parents: List[Stage],
val priority: Int)
extends Logging {
val isShuffleMap = shuffleDep != None
val numPartitions = rdd.splits.size
- val outputLocs = Array.fill[List[BlockManagerId]](numPartitions)(Nil)
+ val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil)
var numAvailableOutputs = 0
private var nextAttemptId = 0
def isAvailable: Boolean = {
- if (/*parents.size == 0 &&*/ !isShuffleMap) {
+ if (!isShuffleMap) {
true
} else {
numAvailableOutputs == numPartitions
}
}
- def addOutputLoc(partition: Int, bmAddress: BlockManagerId) {
+ def addOutputLoc(partition: Int, status: MapStatus) {
val prevList = outputLocs(partition)
- outputLocs(partition) = bmAddress :: prevList
+ outputLocs(partition) = status :: prevList
if (prevList == Nil)
numAvailableOutputs += 1
}
def removeOutputLoc(partition: Int, bmAddress: BlockManagerId) {
val prevList = outputLocs(partition)
- val newList = prevList.filterNot(_ == bmAddress)
+ val newList = prevList.filterNot(_.address == bmAddress)
outputLocs(partition) = newList
if (prevList != Nil && newList == Nil) {
numAvailableOutputs -= 1
@@ -62,7 +62,7 @@ class Stage(
var becameUnavailable = false
for (partition <- 0 until numPartitions) {
val prevList = outputLocs(partition)
- val newList = prevList.filterNot(_.ip == host)
+ val newList = prevList.filterNot(_.address.ip == host)
outputLocs(partition) = newList
if (prevList != Nil && newList == Nil) {
becameUnavailable = true
@@ -80,6 +80,8 @@ class Stage(
return id
}
+ def origin: String = rdd.origin
+
override def toString = "Stage " + id // + ": [RDD = " + rdd.id + ", isShuffle = " + isShuffleMap + "]"
override def hashCode(): Int = id
diff --git a/core/src/main/scala/spark/scheduler/Task.scala b/core/src/main/scala/spark/scheduler/Task.scala
index f84d8d9c4f..ef987fdeb6 100644
--- a/core/src/main/scala/spark/scheduler/Task.scala
+++ b/core/src/main/scala/spark/scheduler/Task.scala
@@ -1,11 +1,95 @@
package spark.scheduler
+import scala.collection.mutable.HashMap
+import spark.serializer.{SerializerInstance, Serializer}
+import java.io.{DataInputStream, DataOutputStream}
+import java.nio.ByteBuffer
+import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
+import spark.util.ByteBufferInputStream
+import scala.collection.mutable.HashMap
+
/**
* A task to execute on a worker node.
*/
-abstract class Task[T](val stageId: Int) extends Serializable {
+private[spark] abstract class Task[T](val stageId: Int) extends Serializable {
def run(attemptId: Long): T
def preferredLocations: Seq[String] = Nil
var generation: Long = -1 // Map output tracker generation. Will be set by TaskScheduler.
}
+
+/**
+ * Handles transmission of tasks and their dependencies, because this can be slightly tricky. We
+ * need to send the list of JARs and files added to the SparkContext with each task to ensure that
+ * worker nodes find out about it, but we can't make it part of the Task because the user's code in
+ * the task might depend on one of the JARs. Thus we serialize each task as multiple objects, by
+ * first writing out its dependencies.
+ */
+private[spark] object Task {
+ /**
+ * Serialize a task and the current app dependencies (files and JARs added to the SparkContext)
+ */
+ def serializeWithDependencies(
+ task: Task[_],
+ currentFiles: HashMap[String, Long],
+ currentJars: HashMap[String, Long],
+ serializer: SerializerInstance)
+ : ByteBuffer = {
+
+ val out = new FastByteArrayOutputStream(4096)
+ val dataOut = new DataOutputStream(out)
+
+ // Write currentFiles
+ dataOut.writeInt(currentFiles.size)
+ for ((name, timestamp) <- currentFiles) {
+ dataOut.writeUTF(name)
+ dataOut.writeLong(timestamp)
+ }
+
+ // Write currentJars
+ dataOut.writeInt(currentJars.size)
+ for ((name, timestamp) <- currentJars) {
+ dataOut.writeUTF(name)
+ dataOut.writeLong(timestamp)
+ }
+
+ // Write the task itself and finish
+ dataOut.flush()
+ val taskBytes = serializer.serialize(task).array()
+ out.write(taskBytes)
+ out.trim()
+ ByteBuffer.wrap(out.array)
+ }
+
+ /**
+ * Deserialize the list of dependencies in a task serialized with serializeWithDependencies,
+ * and return the task itself as a serialized ByteBuffer. The caller can then update its
+ * ClassLoaders and deserialize the task.
+ *
+ * @return (taskFiles, taskJars, taskBytes)
+ */
+ def deserializeWithDependencies(serializedTask: ByteBuffer)
+ : (HashMap[String, Long], HashMap[String, Long], ByteBuffer) = {
+
+ val in = new ByteBufferInputStream(serializedTask)
+ val dataIn = new DataInputStream(in)
+
+ // Read task's files
+ val taskFiles = new HashMap[String, Long]()
+ val numFiles = dataIn.readInt()
+ for (i <- 0 until numFiles) {
+ taskFiles(dataIn.readUTF()) = dataIn.readLong()
+ }
+
+ // Read task's JARs
+ val taskJars = new HashMap[String, Long]()
+ val numJars = dataIn.readInt()
+ for (i <- 0 until numJars) {
+ taskJars(dataIn.readUTF()) = dataIn.readLong()
+ }
+
+ // Create a sub-buffer for the rest of the data, which is the serialized Task object
+ val subBuffer = serializedTask.slice() // ByteBufferInputStream will have read just up to task
+ (taskFiles, taskJars, subBuffer)
+ }
+}
diff --git a/core/src/main/scala/spark/scheduler/TaskResult.scala b/core/src/main/scala/spark/scheduler/TaskResult.scala
index 868ddb237c..9a54d0e854 100644
--- a/core/src/main/scala/spark/scheduler/TaskResult.scala
+++ b/core/src/main/scala/spark/scheduler/TaskResult.scala
@@ -7,6 +7,7 @@ import scala.collection.mutable.Map
// Task result. Also contains updates to accumulator variables.
// TODO: Use of distributed cache to return result is a hack to get around
// what seems to be a bug with messages over 60KB in libprocess; fix it
+private[spark]
class TaskResult[T](var value: T, var accumUpdates: Map[Long, Any]) extends Externalizable {
def this() = this(null.asInstanceOf[T], null)
diff --git a/core/src/main/scala/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/spark/scheduler/TaskScheduler.scala
index c35633d53c..d549b184b0 100644
--- a/core/src/main/scala/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/TaskScheduler.scala
@@ -7,7 +7,7 @@ package spark.scheduler
* are failures, and mitigating stragglers. They return events to the DAGScheduler through
* the TaskSchedulerListener interface.
*/
-trait TaskScheduler {
+private[spark] trait TaskScheduler {
def start(): Unit
// Disconnect from the cluster.
diff --git a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
index a647eec9e4..fa4de15d0d 100644
--- a/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
+++ b/core/src/main/scala/spark/scheduler/TaskSchedulerListener.scala
@@ -7,10 +7,13 @@ import spark.TaskEndReason
/**
* Interface for getting events back from the TaskScheduler.
*/
-trait TaskSchedulerListener {
+private[spark] trait TaskSchedulerListener {
// A task has finished or failed.
def taskEnded(task: Task[_], reason: TaskEndReason, result: Any, accumUpdates: Map[Long, Any]): Unit
// A node was lost from the cluster.
def hostLost(host: String): Unit
+
+ // The TaskScheduler wants to abort an entire task set.
+ def taskSetFailed(taskSet: TaskSet, reason: String): Unit
}
diff --git a/core/src/main/scala/spark/scheduler/TaskSet.scala b/core/src/main/scala/spark/scheduler/TaskSet.scala
index 6f29dd2e9d..a3002ca477 100644
--- a/core/src/main/scala/spark/scheduler/TaskSet.scala
+++ b/core/src/main/scala/spark/scheduler/TaskSet.scala
@@ -4,6 +4,8 @@ package spark.scheduler
* A set of tasks submitted together to the low-level TaskScheduler, usually representing
* missing partitions of a particular stage.
*/
-class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int) {
+private[spark] class TaskSet(val tasks: Array[Task[_]], val stageId: Int, val attempt: Int, val priority: Int) {
val id: String = stageId + "." + attempt
+
+ override def toString: String = "TaskSet " + id
}
diff --git a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
index 20c82ad0fa..f5e852d203 100644
--- a/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/ClusterScheduler.scala
@@ -16,7 +16,7 @@ import java.util.concurrent.atomic.AtomicLong
* The main TaskScheduler implementation, for running tasks on a cluster. Clients should first call
* start(), then submit task sets through the runTasks method.
*/
-class ClusterScheduler(sc: SparkContext)
+private[spark] class ClusterScheduler(val sc: SparkContext)
extends TaskScheduler
with Logging {
@@ -60,7 +60,6 @@ class ClusterScheduler(sc: SparkContext)
def initialize(context: SchedulerBackend) {
backend = context
- createJarServer()
}
def newTaskId(): Long = nextTaskId.getAndIncrement()
@@ -236,32 +235,7 @@ class ClusterScheduler(sc: SparkContext)
}
override def defaultParallelism() = backend.defaultParallelism()
-
- // Create a server for all the JARs added by the user to SparkContext.
- // We first copy the JARs to a temp directory for easier server setup.
- private def createJarServer() {
- val jarDir = Utils.createTempDir()
- logInfo("Temp directory for JARs: " + jarDir)
- val filenames = ArrayBuffer[String]()
- // Copy each JAR to a unique filename in the jarDir
- for ((path, index) <- sc.jars.zipWithIndex) {
- val file = new File(path)
- if (file.exists) {
- val filename = index + "_" + file.getName
- Utils.copyFile(file, new File(jarDir, filename))
- filenames += filename
- }
- }
- // Create the server
- jarServer = new HttpServer(jarDir)
- jarServer.start()
- // Build up the jar URI list
- val serverUri = jarServer.uri
- jarUris = filenames.map(f => serverUri + "/" + f).mkString(",")
- System.setProperty("spark.jar.uris", jarUris)
- logInfo("JAR server started at " + serverUri)
- }
-
+
// Check for speculatable tasks in all our active jobs.
def checkSpeculatableTasks() {
var shouldRevive = false
diff --git a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
index 897976c3f9..ddcd64d7c6 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SchedulerBackend.scala
@@ -5,7 +5,7 @@ package spark.scheduler.cluster
* ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as
* machines become available and can launch tasks on them.
*/
-trait SchedulerBackend {
+private[spark] trait SchedulerBackend {
def start(): Unit
def stop(): Unit
def reviveOffers(): Unit
diff --git a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala b/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala
index e15d577a8b..96ebaa4601 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SlaveResources.scala
@@ -1,3 +1,4 @@
package spark.scheduler.cluster
+private[spark]
class SlaveResources(val slaveId: String, val hostname: String, val coresFree: Int) {}
diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index 0bd2d15479..7aba7324ab 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -5,7 +5,7 @@ import spark.deploy.client.{Client, ClientListener}
import spark.deploy.{Command, JobDescription}
import scala.collection.mutable.HashMap
-class SparkDeploySchedulerBackend(
+private[spark] class SparkDeploySchedulerBackend(
scheduler: ClusterScheduler,
sc: SparkContext,
master: String,
@@ -16,17 +16,10 @@ class SparkDeploySchedulerBackend(
var client: Client = null
var stopping = false
+ var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _
val maxCores = System.getProperty("spark.cores.max", Int.MaxValue.toString).toInt
- // Environment variables to pass to our executors
- val ENV_VARS_TO_SEND_TO_EXECUTORS = Array(
- "SPARK_MEM",
- "SPARK_CLASSPATH",
- "SPARK_LIBRARY_PATH",
- "SPARK_JAVA_OPTS"
- )
-
// Memory used by each executor (in megabytes)
val executorMemory = {
if (System.getenv("SPARK_MEM") != null) {
@@ -40,17 +33,11 @@ class SparkDeploySchedulerBackend(
override def start() {
super.start()
- val environment = new HashMap[String, String]
- for (key <- ENV_VARS_TO_SEND_TO_EXECUTORS) {
- if (System.getenv(key) != null) {
- environment(key) = System.getenv(key)
- }
- }
val masterUrl = "akka://spark@%s:%s/user/%s".format(
System.getProperty("spark.master.host"), System.getProperty("spark.master.port"),
StandaloneSchedulerBackend.ACTOR_NAME)
val args = Seq(masterUrl, "{{SLAVEID}}", "{{HOSTNAME}}", "{{CORES}}")
- val command = Command("spark.executor.StandaloneExecutorBackend", args, environment)
+ val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command)
client = new Client(sc.env.actorSystem, master, jobDesc, this)
@@ -61,6 +48,9 @@ class SparkDeploySchedulerBackend(
stopping = true;
super.stop()
client.stop()
+ if (shutdownCallback != null) {
+ shutdownCallback(this)
+ }
}
def connected(jobId: String) {
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
index 80e8733671..1386cd9d44 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneClusterMessage.scala
@@ -4,19 +4,27 @@ import spark.TaskState.TaskState
import java.nio.ByteBuffer
import spark.util.SerializableBuffer
-sealed trait StandaloneClusterMessage extends Serializable
+private[spark] sealed trait StandaloneClusterMessage extends Serializable
// Master to slaves
+private[spark]
case class LaunchTask(task: TaskDescription) extends StandaloneClusterMessage
+
+private[spark]
case class RegisteredSlave(sparkProperties: Seq[(String, String)]) extends StandaloneClusterMessage
+
+private[spark]
case class RegisterSlaveFailed(message: String) extends StandaloneClusterMessage
// Slaves to master
+private[spark]
case class RegisterSlave(slaveId: String, host: String, cores: Int) extends StandaloneClusterMessage
+private[spark]
case class StatusUpdate(slaveId: String, taskId: Long, state: TaskState, data: SerializableBuffer)
extends StandaloneClusterMessage
+private[spark]
object StatusUpdate {
/** Alternate factory method that takes a ByteBuffer directly for the data field */
def apply(slaveId: String, taskId: Long, state: TaskState, data: ByteBuffer): StatusUpdate = {
@@ -25,5 +33,5 @@ object StatusUpdate {
}
// Internal messages in master
-case object ReviveOffers extends StandaloneClusterMessage
-case object StopMaster extends StandaloneClusterMessage
+private[spark] case object ReviveOffers extends StandaloneClusterMessage
+private[spark] case object StopMaster extends StandaloneClusterMessage
diff --git a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
index 978b4f2676..d2cce0dc05 100644
--- a/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
@@ -16,6 +16,7 @@ import akka.remote.{RemoteClientShutdown, RemoteClientDisconnected, RemoteClient
* Akka. These may be executed in a variety of ways, such as Mesos tasks for the coarse-grained
* Mesos mode or standalone processes for Spark's standalone deploy mode (spark.deploy.*).
*/
+private[spark]
class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: ActorSystem)
extends SchedulerBackend with Logging {
@@ -99,7 +100,7 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
// Remove a disconnected slave from the cluster
def removeSlave(slaveId: String) {
- logWarning("Slave " + slaveId + " disconnected, so removing it")
+ logInfo("Slave " + slaveId + " disconnected, so removing it")
val numCores = freeCores(slaveId)
actorToSlaveId -= slaveActor(slaveId)
addressToSlaveId -= slaveAddress(slaveId)
@@ -149,6 +150,6 @@ class StandaloneSchedulerBackend(scheduler: ClusterScheduler, actorSystem: Actor
def defaultParallelism(): Int = math.max(totalCoreCount.get(), 2)
}
-object StandaloneSchedulerBackend {
+private[spark] object StandaloneSchedulerBackend {
val ACTOR_NAME = "StandaloneScheduler"
}
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
index f9a1b74fa5..aa097fd3a2 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskDescription.scala
@@ -3,7 +3,7 @@ package spark.scheduler.cluster
import java.nio.ByteBuffer
import spark.util.SerializableBuffer
-class TaskDescription(
+private[spark] class TaskDescription(
val taskId: Long,
val slaveId: String,
val name: String,
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
index 65e59841a9..ca84503780 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskInfo.scala
@@ -3,6 +3,7 @@ package spark.scheduler.cluster
/**
* Information about a running task attempt inside a TaskSet.
*/
+private[spark]
class TaskInfo(val taskId: Long, val index: Int, val launchTime: Long, val host: String) {
var finishTime: Long = 0
var failed = false
diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
index 5a7df6040c..cf4aae03a7 100644
--- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala
@@ -17,7 +17,7 @@ import java.nio.ByteBuffer
/**
* Schedules the tasks within a single TaskSet in the ClusterScheduler.
*/
-class TaskSetManager(
+private[spark] class TaskSetManager(
sched: ClusterScheduler,
val taskSet: TaskSet)
extends Logging {
@@ -214,7 +214,8 @@ class TaskSetManager(
}
// Serialize and return the task
val startTime = System.currentTimeMillis
- val serializedTask = ser.serialize(task)
+ val serializedTask = Task.serializeWithDependencies(
+ task, sched.sc.addedFiles, sched.sc.addedJars, ser)
val timeTaken = System.currentTimeMillis - startTime
logInfo("Serialized task %s:%d as %d bytes in %d ms".format(
taskSet.id, index, serializedTask.limit, timeTaken))
@@ -243,6 +244,11 @@ class TaskSetManager(
def taskFinished(tid: Long, state: TaskState, serializedData: ByteBuffer) {
val info = taskInfos(tid)
+ if (info.failed) {
+ // We might get two task-lost messages for the same task in coarse-grained Mesos mode,
+ // or even from Mesos itself when acks get delayed.
+ return
+ }
val index = info.index
info.markSuccessful()
if (!finished(index)) {
@@ -335,13 +341,14 @@ class TaskSetManager(
def error(message: String) {
// Save the error message
- abort("Mesos 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.listener.taskSetFailed(taskSet, message)
sched.taskSetFinished(this)
}
diff --git a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
index 1e83f103e7..6b919d68b2 100644
--- a/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/WorkerOffer.scala
@@ -3,5 +3,6 @@ package spark.scheduler.cluster
/**
* Represents free resources available on a worker node.
*/
+private[spark]
class WorkerOffer(val slaveId: String, val hostname: String, val cores: Int) {
}
diff --git a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
index eb47988f0c..eb20fe41b2 100644
--- a/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/local/LocalScheduler.scala
@@ -1,9 +1,13 @@
package spark.scheduler.local
+import java.io.File
+import java.net.URLClassLoader
import java.util.concurrent.Executors
import java.util.concurrent.atomic.AtomicInteger
+import scala.collection.mutable.HashMap
import spark._
+import executor.ExecutorURLClassLoader
import spark.scheduler._
/**
@@ -11,17 +15,27 @@ import spark.scheduler._
* the scheduler also allows each task to fail up to maxFailures times, which is useful for
* testing fault recovery.
*/
-class LocalScheduler(threads: Int, maxFailures: Int) extends TaskScheduler with Logging {
+private[spark] class LocalScheduler(threads: Int, maxFailures: Int, sc: SparkContext)
+ extends TaskScheduler
+ with Logging {
+
var attemptId = new AtomicInteger(0)
var threadPool = Executors.newFixedThreadPool(threads, DaemonThreadFactory)
val env = SparkEnv.get
var listener: TaskSchedulerListener = 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]()
+
+ val classLoader = new ExecutorURLClassLoader(Array(), Thread.currentThread.getContextClassLoader)
+
// TODO: Need to take into account stage priority in scheduling
- override def start() {}
+ override def start() { }
- override def setListener(listener: TaskSchedulerListener) {
+ override def setListener(listener: TaskSchedulerListener) {
this.listener = listener
}
@@ -43,21 +57,29 @@ class LocalScheduler(threads: Int, maxFailures: Int) extends TaskScheduler with
// Set the Spark execution environment for the worker thread
SparkEnv.set(env)
try {
+ Accumulators.clear()
+ Thread.currentThread().setContextClassLoader(classLoader)
+
// Serialize and deserialize the task so that accumulators are changed to thread-local ones;
// this adds a bit of unnecessary overhead but matches how the Mesos Executor works.
- Accumulators.clear
val ser = SparkEnv.get.closureSerializer.newInstance()
- val bytes = ser.serialize(task)
+ val bytes = Task.serializeWithDependencies(task, sc.addedFiles, sc.addedJars, ser)
logInfo("Size of task " + idInJob + " is " + bytes.limit + " bytes")
+ val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(bytes)
+ updateDependencies(taskFiles, taskJars) // Download any files added with addFile
val deserializedTask = ser.deserialize[Task[_]](
- bytes, Thread.currentThread.getContextClassLoader)
+ taskBytes, Thread.currentThread.getContextClassLoader)
+
+ // Run it
val result: Any = deserializedTask.run(attemptId)
+
// Serialize and deserialize the result to emulate what the Mesos
// executor does. This is useful to catch serialization errors early
// on in development (so when users move their local Spark programs
// to the cluster, they don't get surprised by serialization errors).
val resultToReturn = ser.deserialize[Any](ser.serialize(result))
- val accumUpdates = Accumulators.values
+ val accumUpdates = ser.deserialize[collection.mutable.Map[Long, Any]](
+ ser.serialize(Accumulators.values))
logInfo("Finished task " + idInJob)
listener.taskEnded(task, Success, resultToReturn, accumUpdates)
} catch {
@@ -80,7 +102,32 @@ class LocalScheduler(threads: Int, maxFailures: Int) extends TaskScheduler with
submitTask(task, i)
}
}
-
+
+ /**
+ * Download any missing dependencies if we receive a new set of files and JARs from the
+ * SparkContext. Also adds any new JARs we fetched to the class loader.
+ */
+ private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) {
+ // Fetch missing dependencies
+ for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) {
+ logInfo("Fetching " + name + " with timestamp " + timestamp)
+ Utils.fetchFile(name, new File("."))
+ currentFiles(name) = timestamp
+ }
+ for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) {
+ logInfo("Fetching " + name + " with timestamp " + timestamp)
+ Utils.fetchFile(name, new File("."))
+ currentJars(name) = timestamp
+ // Add it to our class loader
+ val localName = name.split("/").last
+ val url = new File(".", localName).toURI.toURL
+ if (!classLoader.getURLs.contains(url)) {
+ logInfo("Adding " + url + " to class loader")
+ classLoader.addURL(url)
+ }
+ }
+ }
+
override def stop() {
threadPool.shutdownNow()
}
diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
index fdf007ffb2..c45c7df69c 100644
--- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
@@ -24,7 +24,7 @@ import spark.TaskState
* Unfortunately this has a bit of duplication from MesosSchedulerBackend, but it seems hard to
* remove this.
*/
-class CoarseMesosSchedulerBackend(
+private[spark] class CoarseMesosSchedulerBackend(
scheduler: ClusterScheduler,
sc: SparkContext,
master: String,
@@ -33,14 +33,6 @@ class CoarseMesosSchedulerBackend(
with MScheduler
with Logging {
- // Environment variables to pass to our executors
- val ENV_VARS_TO_SEND_TO_EXECUTORS = Array(
- "SPARK_MEM",
- "SPARK_CLASSPATH",
- "SPARK_LIBRARY_PATH",
- "SPARK_JAVA_OPTS"
- )
-
val MAX_SLAVE_FAILURES = 2 // Blacklist a slave after this many failures
// Memory used by each executor (in megabytes)
@@ -122,13 +114,11 @@ class CoarseMesosSchedulerBackend(
val command = "\"%s\" spark.executor.StandaloneExecutorBackend %s %s %s %d".format(
runScript, masterUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)
val environment = Environment.newBuilder()
- for (key <- ENV_VARS_TO_SEND_TO_EXECUTORS) {
- if (System.getenv(key) != null) {
- environment.addVariables(Environment.Variable.newBuilder()
- .setName(key)
- .setValue(System.getenv(key))
- .build())
- }
+ sc.executorEnvs.foreach { case (key, value) =>
+ environment.addVariables(Environment.Variable.newBuilder()
+ .setName(key)
+ .setValue(value)
+ .build())
}
return CommandInfo.newBuilder().setValue(command).setEnvironment(environment).build()
}
diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
index 44eda93dd1..cdfe1f2563 100644
--- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
@@ -20,7 +20,7 @@ import spark.TaskState
* separate Mesos task, allowing multiple applications to share cluster nodes both in space (tasks
* from multiple apps can run on different cores) and in time (a core can switch ownership).
*/
-class MesosSchedulerBackend(
+private[spark] class MesosSchedulerBackend(
scheduler: ClusterScheduler,
sc: SparkContext,
master: String,
@@ -29,14 +29,6 @@ class MesosSchedulerBackend(
with MScheduler
with Logging {
- // Environment variables to pass to our executors
- val ENV_VARS_TO_SEND_TO_EXECUTORS = Array(
- "SPARK_MEM",
- "SPARK_CLASSPATH",
- "SPARK_LIBRARY_PATH",
- "SPARK_JAVA_OPTS"
- )
-
// Memory used by each executor (in megabytes)
val EXECUTOR_MEMORY = {
if (System.getenv("SPARK_MEM") != null) {
@@ -93,13 +85,11 @@ class MesosSchedulerBackend(
}
val execScript = new File(sparkHome, "spark-executor").getCanonicalPath
val environment = Environment.newBuilder()
- for (key <- ENV_VARS_TO_SEND_TO_EXECUTORS) {
- if (System.getenv(key) != null) {
- environment.addVariables(Environment.Variable.newBuilder()
- .setName(key)
- .setValue(System.getenv(key))
- .build())
- }
+ sc.executorEnvs.foreach { case (key, value) =>
+ environment.addVariables(Environment.Variable.newBuilder()
+ .setName(key)
+ .setValue(value)
+ .build())
}
val memory = Resource.newBuilder()
.setName("mem")
diff --git a/core/src/main/scala/spark/Serializer.scala b/core/src/main/scala/spark/serializer/Serializer.scala
index 61a70beaf1..50b086125a 100644
--- a/core/src/main/scala/spark/Serializer.scala
+++ b/core/src/main/scala/spark/serializer/Serializer.scala
@@ -1,23 +1,21 @@
-package spark
+package spark.serializer
-import java.io.{EOFException, InputStream, OutputStream}
import java.nio.ByteBuffer
-import java.nio.channels.Channels
-
+import java.io.{EOFException, InputStream, OutputStream}
import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
-
import spark.util.ByteBufferInputStream
/**
- * A serializer. Because some serialization libraries are not thread safe, this class is used to
- * create SerializerInstances that do the actual serialization.
+ * A serializer. Because some serialization libraries are not thread safe, this class is used to
+ * create [[spark.serializer.SerializerInstance]] objects that do the actual serialization and are
+ * guaranteed to only be called from one thread at a time.
*/
trait Serializer {
def newInstance(): SerializerInstance
}
/**
- * An instance of the serializer, for use by one thread at a time.
+ * An instance of a serializer, for use by one thread at a time.
*/
trait SerializerInstance {
def serialize[T](t: T): ByteBuffer
@@ -43,7 +41,7 @@ trait SerializerInstance {
def deserializeMany(buffer: ByteBuffer): Iterator[Any] = {
// Default implementation uses deserializeStream
buffer.rewind()
- deserializeStream(new ByteBufferInputStream(buffer)).toIterator
+ deserializeStream(new ByteBufferInputStream(buffer)).asIterator
}
}
@@ -51,7 +49,7 @@ trait SerializerInstance {
* A stream for writing serialized objects.
*/
trait SerializationStream {
- def writeObject[T](t: T): Unit
+ def writeObject[T](t: T): SerializationStream
def flush(): Unit
def close(): Unit
@@ -74,7 +72,7 @@ trait DeserializationStream {
* Read the elements of this stream through an iterator. This can only be called once, as
* reading each element will consume data from the input source.
*/
- def toIterator: Iterator[Any] = new Iterator[Any] {
+ def asIterator: Iterator[Any] = new Iterator[Any] {
var gotNext = false
var finished = false
var nextValue: Any = null
@@ -88,7 +86,7 @@ trait DeserializationStream {
}
gotNext = true
}
-
+
override def hasNext: Boolean = {
if (!gotNext) {
getNext()
diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala
index 4cdb9710ec..bd9155ef29 100644
--- a/core/src/main/scala/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/spark/storage/BlockManager.scala
@@ -1,34 +1,29 @@
package spark.storage
-import java.io._
-import java.nio._
-import java.nio.channels.FileChannel.MapMode
-import java.util.{HashMap => JHashMap}
-import java.util.LinkedHashMap
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.LinkedBlockingQueue
-import java.util.Collections
-
import akka.dispatch.{Await, Future}
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.JavaConversions._
+import akka.util.Duration
-import it.unimi.dsi.fastutil.io._
+import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream
-import spark.CacheTracker
-import spark.Logging
-import spark.Serializer
-import spark.SizeEstimator
-import spark.SparkEnv
-import spark.SparkException
-import spark.Utils
-import spark.util.ByteBufferInputStream
+import java.io.{InputStream, OutputStream, Externalizable, ObjectInput, ObjectOutput}
+import java.nio.{MappedByteBuffer, ByteBuffer}
+import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue}
+
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
+import scala.collection.JavaConversions._
+
+import spark.{CacheTracker, Logging, SizeEstimator, SparkException, Utils}
import spark.network._
-import akka.util.Duration
+import spark.serializer.Serializer
+import spark.util.ByteBufferInputStream
+import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream}
+import sun.nio.ch.DirectBuffer
+
-class BlockManagerId(var ip: String, var port: Int) extends Externalizable {
- def this() = this(null, 0)
+private[spark] class BlockManagerId(var ip: String, var port: Int) extends Externalizable {
+ def this() = this(null, 0) // For deserialization only
+
+ def this(in: ObjectInput) = this(in.readUTF(), in.readInt())
override def writeExternal(out: ObjectOutput) {
out.writeUTF(ip)
@@ -51,41 +46,76 @@ class BlockManagerId(var ip: String, var port: Int) extends Externalizable {
}
-case class BlockException(blockId: String, message: String, ex: Exception = null) extends Exception(message)
+private[spark]
+case class BlockException(blockId: String, message: String, ex: Exception = null)
+extends Exception(message)
-class BlockLocker(numLockers: Int) {
+private[spark] class BlockLocker(numLockers: Int) {
private val hashLocker = Array.fill(numLockers)(new Object())
-
+
def getLock(blockId: String): Object = {
return hashLocker(math.abs(blockId.hashCode % numLockers))
}
}
+private[spark]
class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, maxMemory: Long)
extends Logging {
- case class BlockInfo(level: StorageLevel, tellMaster: Boolean)
+ class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) {
+ var pending: Boolean = true
+ var size: Long = -1L
+
+ /** Wait for this BlockInfo to be marked as ready (i.e. block is finished writing) */
+ def waitForReady() {
+ if (pending) {
+ synchronized {
+ while (pending) this.wait()
+ }
+ }
+ }
+
+ /** Mark this BlockInfo as ready (i.e. block is finished writing) */
+ def markReady(sizeInBytes: Long) {
+ pending = false
+ size = sizeInBytes
+ synchronized {
+ this.notifyAll()
+ }
+ }
+ }
private val NUM_LOCKS = 337
private val locker = new BlockLocker(NUM_LOCKS)
private val blockInfo = new ConcurrentHashMap[String, BlockInfo]()
- private val memoryStore: BlockStore = new MemoryStore(this, maxMemory)
- private val diskStore: BlockStore = new DiskStore(this,
- System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
-
+
+ private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory)
+ private[storage] val diskStore: BlockStore =
+ new DiskStore(this, System.getProperty("spark.local.dir", System.getProperty("java.io.tmpdir")))
+
val connectionManager = new ConnectionManager(0)
implicit val futureExecContext = connectionManager.futureExecContext
-
+
val connectionManagerId = connectionManager.id
val blockManagerId = new BlockManagerId(connectionManagerId.host, connectionManagerId.port)
-
+
// TODO: This will be removed after cacheTracker is removed from the code base.
var cacheTracker: CacheTracker = null
- initLogging()
+ // 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
+
+ val compressBroadcast = System.getProperty("spark.broadcast.compress", "true").toBoolean
+ val compressShuffle = System.getProperty("spark.shuffle.compress", "true").toBoolean
+ // Whether to compress RDD partitions that are stored serialized
+ val compressRdds = System.getProperty("spark.rdd.compress", "false").toBoolean
+
+ val host = System.getProperty("spark.hostname", Utils.localHostName())
initialize()
@@ -102,7 +132,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
*/
private def initialize() {
master.mustRegisterBlockManager(
- RegisterBlockManager(blockManagerId, maxMemory, maxMemory))
+ RegisterBlockManager(blockManagerId, maxMemory))
BlockManagerWorker.startBlockManagerWorker(this)
}
@@ -115,36 +145,32 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
}
/**
- * Change storage level for a local block and tell master is necesary.
- * If new level is invalid, then block info (if it exists) will be silently removed.
+ * Tell the master about the current storage status of a block. This will send a heartbeat
+ * message reflecting the current status, *not* the desired storage level in its block info.
+ * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk.
*/
- def setLevel(blockId: String, level: StorageLevel, tellMaster: Boolean = true) {
- if (level == null) {
- throw new IllegalArgumentException("Storage level is null")
- }
-
- // If there was earlier info about the block, then use earlier tellMaster
- val oldInfo = blockInfo.get(blockId)
- val newTellMaster = if (oldInfo != null) oldInfo.tellMaster else tellMaster
- if (oldInfo != null && oldInfo.tellMaster != tellMaster) {
- logWarning("Ignoring tellMaster setting as it is different from earlier setting")
- }
-
- // If level is valid, store the block info, else remove the block info
- if (level.isValid) {
- blockInfo.put(blockId, new BlockInfo(level, newTellMaster))
- logDebug("Info for block " + blockId + " updated with new level as " + level)
- } else {
- blockInfo.remove(blockId)
- logDebug("Info for block " + blockId + " removed as new level is null or invalid")
- }
-
- // Tell master if necessary
- if (newTellMaster) {
+ def reportBlockStatus(blockId: String) {
+ locker.getLock(blockId).synchronized {
+ val curLevel = blockInfo.get(blockId) match {
+ case null =>
+ StorageLevel.NONE
+ case info =>
+ info.level match {
+ case null =>
+ StorageLevel.NONE
+ case level =>
+ val inMem = level.useMemory && memoryStore.contains(blockId)
+ val onDisk = level.useDisk && diskStore.contains(blockId)
+ new StorageLevel(onDisk, inMem, level.deserialized, level.replication)
+ }
+ }
+ master.mustHeartBeat(HeartBeat(
+ blockManagerId,
+ blockId,
+ curLevel,
+ if (curLevel.useMemory) memoryStore.getSize(blockId) else 0L,
+ if (curLevel.useDisk) diskStore.getSize(blockId) else 0L))
logDebug("Told master about block " + blockId)
- notifyMaster(HeartBeat(blockManagerId, blockId, level, 0, 0))
- } else {
- logDebug("Did not tell master about block " + blockId)
}
}
@@ -174,55 +200,149 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
* Get block from local block manager.
*/
def getLocal(blockId: String): Option[Iterator[Any]] = {
- if (blockId == null) {
- throw new IllegalArgumentException("Block Id is null")
- }
logDebug("Getting local block " + blockId)
+
+ // As an optimization for map output fetches, if the block is for a shuffle, return it
+ // without acquiring a lock; the disk store never deletes (recent) items so this should work
+ if (blockId.startsWith("shuffle_")) {
+ return diskStore.getValues(blockId) match {
+ case Some(iterator) =>
+ Some(iterator)
+ case None =>
+ throw new Exception("Block " + blockId + " not found on disk, though it should be")
+ }
+ }
+
locker.getLock(blockId).synchronized {
-
- // Check storage level of block
- val level = getLevel(blockId)
- if (level != null) {
- logDebug("Level for block " + blockId + " is " + level + " on local machine")
-
+ val info = blockInfo.get(blockId)
+ if (info != null) {
+ info.waitForReady() // In case the block is still being put() by another thread
+ val level = info.level
+ logDebug("Level for block " + blockId + " is " + level)
+
// Look for the block in memory
if (level.useMemory) {
logDebug("Getting block " + blockId + " from memory")
memoryStore.getValues(blockId) match {
- case Some(iterator) => {
- logDebug("Block " + blockId + " found in memory")
+ case Some(iterator) =>
return Some(iterator)
- }
- case None => {
+ case None =>
logDebug("Block " + blockId + " not found in memory")
- }
}
- } else {
- logDebug("Not getting block " + blockId + " from memory")
}
- // Look for block in disk
+ // Look for block on disk, potentially loading it back into memory if required
if (level.useDisk) {
logDebug("Getting block " + blockId + " from disk")
- diskStore.getValues(blockId) match {
- case Some(iterator) => {
- logDebug("Block " + blockId + " found in disk")
- return Some(iterator)
+ if (level.useMemory && level.deserialized) {
+ diskStore.getValues(blockId) match {
+ case Some(iterator) =>
+ // Put the block back in memory before returning it
+ // TODO: Consider creating a putValues that also takes in a iterator ?
+ val elements = new ArrayBuffer[Any]
+ elements ++= iterator
+ memoryStore.putValues(blockId, elements, level, true).data match {
+ case Left(iterator2) =>
+ return Some(iterator2)
+ case _ =>
+ throw new Exception("Memory store did not return back an iterator")
+ }
+ case None =>
+ throw new Exception("Block " + blockId + " not found on disk, though it should be")
+ }
+ } else if (level.useMemory && !level.deserialized) {
+ // Read it as a byte buffer into memory first, then return it
+ diskStore.getBytes(blockId) match {
+ case Some(bytes) =>
+ // Put a copy of the block back in memory before returning it. Note that we can't
+ // put the ByteBuffer returned by the disk store as that's a memory-mapped file.
+ val copyForMemory = ByteBuffer.allocate(bytes.limit)
+ copyForMemory.put(bytes)
+ memoryStore.putBytes(blockId, copyForMemory, level)
+ bytes.rewind()
+ return Some(dataDeserialize(blockId, bytes))
+ case None =>
+ throw new Exception("Block " + blockId + " not found on disk, though it should be")
}
- case None => {
- throw new Exception("Block " + blockId + " not found in disk")
- return None
+ } else {
+ diskStore.getValues(blockId) match {
+ case Some(iterator) =>
+ return Some(iterator)
+ case None =>
+ throw new Exception("Block " + blockId + " not found on disk, though it should be")
}
}
- } else {
- logDebug("Not getting block " + blockId + " from disk")
}
+ } else {
+ logDebug("Block " + blockId + " not registered locally")
+ }
+ }
+ return None
+ }
+
+ /**
+ * Get block from the local block manager as serialized bytes.
+ */
+ def getLocalBytes(blockId: String): Option[ByteBuffer] = {
+ // TODO: This whole thing is very similar to getLocal; we need to refactor it somehow
+ logDebug("Getting local block " + blockId + " as bytes")
+
+ // As an optimization for map output fetches, if the block is for a shuffle, return it
+ // without acquiring a lock; the disk store never deletes (recent) items so this should work
+ if (blockId.startsWith("shuffle_")) {
+ return diskStore.getBytes(blockId) match {
+ case Some(bytes) =>
+ Some(bytes)
+ case None =>
+ throw new Exception("Block " + blockId + " not found on disk, though it should be")
+ }
+ }
+
+ locker.getLock(blockId).synchronized {
+ val info = blockInfo.get(blockId)
+ if (info != null) {
+ info.waitForReady() // In case the block is still being put() by another thread
+ val level = info.level
+ logDebug("Level for block " + blockId + " is " + level)
+ // Look for the block in memory
+ if (level.useMemory) {
+ logDebug("Getting block " + blockId + " from memory")
+ memoryStore.getBytes(blockId) match {
+ case Some(bytes) =>
+ return Some(bytes)
+ case None =>
+ logDebug("Block " + blockId + " not found in memory")
+ }
+ }
+
+ // Look for block on disk
+ if (level.useDisk) {
+ // Read it as a byte buffer into memory first, then return it
+ diskStore.getBytes(blockId) match {
+ case Some(bytes) =>
+ if (level.useMemory) {
+ if (level.deserialized) {
+ memoryStore.putBytes(blockId, bytes, level)
+ } else {
+ // The memory store will hang onto the ByteBuffer, so give it a copy instead of
+ // the memory-mapped file buffer we got from the disk store
+ val copyForMemory = ByteBuffer.allocate(bytes.limit)
+ copyForMemory.put(bytes)
+ memoryStore.putBytes(blockId, copyForMemory, level)
+ }
+ }
+ bytes.rewind()
+ return Some(bytes)
+ case None =>
+ throw new Exception("Block " + blockId + " not found on disk, though it should be")
+ }
+ }
} else {
- logDebug("Level for block " + blockId + " not found")
+ logDebug("Block " + blockId + " not registered locally")
}
- }
- return None
+ }
+ return None
}
/**
@@ -243,7 +363,7 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
GetBlock(blockId), ConnectionManagerId(loc.ip, loc.port))
if (data != null) {
logDebug("Data is not null: " + data)
- return Some(dataDeserialize(data))
+ return Some(dataDeserialize(blockId, data))
}
logDebug("Data is null")
}
@@ -261,9 +381,10 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
/**
* Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns
* an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined
- * fashion as they're received.
+ * fashion as they're received. Expects a size in bytes to be provided for each block fetched,
+ * so that we can control the maxMegabytesInFlight for the fetch.
*/
- def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[String])])
+ def getMultiple(blocksByAddress: Seq[(BlockManagerId, Seq[(String, Long)])])
: Iterator[(String, Option[Iterator[Any]])] = {
if (blocksByAddress == null) {
@@ -273,71 +394,127 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
logDebug("Getting " + totalBlocks + " blocks")
var startTime = System.currentTimeMillis
val localBlockIds = new ArrayBuffer[String]()
- val remoteBlockIds = new ArrayBuffer[String]()
- val remoteBlockIdsPerLocation = new HashMap[BlockManagerId, Seq[String]]()
+ val remoteBlockIds = new HashSet[String]()
- // A queue to hold our results. Because we want all the deserializing the happen in the
- // caller's thread, this will actually hold functions to produce the Iterator for each block.
- // For local blocks we'll have an iterator already, while for remote ones we'll deserialize.
- val results = new LinkedBlockingQueue[(String, Option[() => Iterator[Any]])]
+ // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize
+ // the block (since we want all deserializaton to happen in the calling thread); can also
+ // represent a fetch failure if size == -1.
+ class FetchResult(val blockId: String, val size: Long, val deserialize: () => Iterator[Any]) {
+ def failed: Boolean = size == -1
+ }
- // Split local and remote blocks
- for ((address, blockIds) <- blocksByAddress) {
- if (address == blockManagerId) {
- localBlockIds ++= blockIds
- } else {
- remoteBlockIds ++= blockIds
- remoteBlockIdsPerLocation(address) = blockIds
- }
+ // A queue to hold our results.
+ val results = new LinkedBlockingQueue[FetchResult]
+
+ // A request to fetch one or more blocks, complete with their sizes
+ class FetchRequest(val address: BlockManagerId, val blocks: Seq[(String, Long)]) {
+ val size = blocks.map(_._2).sum
}
-
- // Start getting remote blocks
- for ((bmId, bIds) <- remoteBlockIdsPerLocation) {
- val cmId = ConnectionManagerId(bmId.ip, bmId.port)
- val blockMessages = bIds.map(bId => BlockMessage.fromGetBlock(GetBlock(bId)))
- val blockMessageArray = new BlockMessageArray(blockMessages)
+
+ // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that
+ // the number of bytes in flight is limited to maxBytesInFlight
+ val fetchRequests = new Queue[FetchRequest]
+
+ // Current bytes in flight from our requests
+ var bytesInFlight = 0L
+
+ def sendRequest(req: FetchRequest) {
+ logDebug("Sending request for %d blocks (%s) from %s".format(
+ req.blocks.size, Utils.memoryBytesToString(req.size), req.address.ip))
+ val cmId = new ConnectionManagerId(req.address.ip, req.address.port)
+ val blockMessageArray = new BlockMessageArray(req.blocks.map {
+ case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId))
+ })
+ bytesInFlight += req.size
+ val sizeMap = req.blocks.toMap // so we can look up the size of each blockID
val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage)
future.onSuccess {
case Some(message) => {
val bufferMessage = message.asInstanceOf[BufferMessage]
val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage)
- blockMessageArray.foreach(blockMessage => {
+ for (blockMessage <- blockMessageArray) {
if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) {
throw new SparkException(
"Unexpected message " + blockMessage.getType + " received from " + cmId)
}
val blockId = blockMessage.getId
- results.put((blockId, Some(() => dataDeserialize(blockMessage.getData))))
+ results.put(new FetchResult(
+ blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData)))
logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime))
- })
+ }
}
case None => {
- logError("Could not get blocks from " + cmId)
- for (blockId <- bIds) {
- results.put((blockId, None))
+ logError("Could not get block(s) from " + cmId)
+ for ((blockId, size) <- req.blocks) {
+ results.put(new FetchResult(blockId, -1, null))
+ }
+ }
+ }
+ }
+
+ // Split local and remote blocks. Remote blocks are further split into FetchRequests of size
+ // at most maxBytesInFlight in order to limit the amount of data in flight.
+ val remoteRequests = new ArrayBuffer[FetchRequest]
+ for ((address, blockInfos) <- blocksByAddress) {
+ if (address == blockManagerId) {
+ localBlockIds ++= blockInfos.map(_._1)
+ } else {
+ remoteBlockIds ++= blockInfos.map(_._1)
+ // Make our requests at least maxBytesInFlight / 5 in length; the reason to keep them
+ // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5
+ // nodes, rather than blocking on reading output from one node.
+ val minRequestSize = math.max(maxBytesInFlight / 5, 1L)
+ logInfo("maxBytesInFlight: " + maxBytesInFlight + ", minRequest: " + minRequestSize)
+ val iterator = blockInfos.iterator
+ var curRequestSize = 0L
+ var curBlocks = new ArrayBuffer[(String, Long)]
+ while (iterator.hasNext) {
+ val (blockId, size) = iterator.next()
+ curBlocks += ((blockId, size))
+ curRequestSize += size
+ if (curRequestSize >= minRequestSize) {
+ // Add this FetchRequest
+ remoteRequests += new FetchRequest(address, curBlocks)
+ curRequestSize = 0
+ curBlocks = new ArrayBuffer[(String, Long)]
}
}
+ // Add in the final request
+ if (!curBlocks.isEmpty) {
+ remoteRequests += new FetchRequest(address, curBlocks)
+ }
}
}
- logDebug("Started remote gets for " + remoteBlockIds.size + " blocks in " +
- Utils.getUsedTimeMs(startTime) + " ms")
+ // Add the remote requests into our queue in a random order
+ fetchRequests ++= Utils.randomize(remoteRequests)
- // Get the local blocks while remote blocks are being fetched
+ // Send out initial requests for blocks, up to our maxBytesInFlight
+ while (!fetchRequests.isEmpty &&
+ (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
+ sendRequest(fetchRequests.dequeue())
+ }
+
+ val numGets = remoteBlockIds.size - fetchRequests.size
+ logInfo("Started " + numGets + " remote gets in " + Utils.getUsedTimeMs(startTime))
+
+ // Get the local blocks while remote blocks are being fetched. Note that it's okay to do
+ // these all at once because they will just memory-map some files, so they won't consume
+ // any memory that might exceed our maxBytesInFlight
startTime = System.currentTimeMillis
- localBlockIds.foreach(id => {
+ for (id <- localBlockIds) {
getLocal(id) match {
- case Some(block) => {
- results.put((id, Some(() => block)))
+ case Some(iter) => {
+ results.put(new FetchResult(id, 0, () => iter)) // Pass 0 as size since it's not in flight
logDebug("Got local block " + id)
}
case None => {
throw new BlockException(id, "Could not get block " + id + " from local machine")
}
}
- })
+ }
logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms")
- // Return an iterator that will read fetched blocks off the queue as they arrive
+ // Return an iterator that will read fetched blocks off the queue as they arrive.
return new Iterator[(String, Option[Iterator[Any]])] {
var resultsGotten = 0
@@ -345,16 +522,30 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
def next(): (String, Option[Iterator[Any]]) = {
resultsGotten += 1
- val (blockId, functionOption) = results.take()
- (blockId, functionOption.map(_.apply()))
+ val result = results.take()
+ bytesInFlight -= result.size
+ if (!fetchRequests.isEmpty &&
+ (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
+ sendRequest(fetchRequests.dequeue())
+ }
+ (result.blockId, if (result.failed) None else Some(result.deserialize()))
}
}
}
+ def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean)
+ : Long = {
+ val elements = new ArrayBuffer[Any]
+ elements ++= values
+ put(blockId, elements, level, tellMaster)
+ }
+
/**
- * Put a new block of values to the block manager.
+ * Put a new block of values to the block manager. Returns its (estimated) size in bytes.
*/
- def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean = true) {
+ def put(blockId: String, values: ArrayBuffer[Any], level: StorageLevel,
+ tellMaster: Boolean = true) : Long = {
+
if (blockId == null) {
throw new IllegalArgumentException("Block Id is null")
}
@@ -365,81 +556,97 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
throw new IllegalArgumentException("Storage level is null or invalid")
}
- val startTimeMs = System.currentTimeMillis
- var bytes: ByteBuffer = null
+ val oldBlock = blockInfo.get(blockId)
+ if (oldBlock != null) {
+ logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
+ oldBlock.waitForReady()
+ return oldBlock.size
+ }
+
+ // Remember the block's storage level so that we can correctly drop it to disk if it needs
+ // to be dropped right after it got put into memory. Note, however, that other threads will
+ // not be able to get() this block until we call markReady on its BlockInfo.
+ val myInfo = new BlockInfo(level, tellMaster)
+ blockInfo.put(blockId, myInfo)
+
+ val startTimeMs = System.currentTimeMillis
// If we need to replicate the data, we'll want access to the values, but because our
// put will read the whole iterator, there will be no values left. For the case where
- // the put serializes data, we'll remember the bytes, above; but for the case where
- // it doesn't, such as MEMORY_ONLY_DESER, let's rely on the put returning an Iterator.
+ // the put serializes data, we'll remember the bytes, above; but for the case where it
+ // doesn't, such as deserialized storage, let's rely on the put returning an Iterator.
var valuesAfterPut: Iterator[Any] = null
-
+
+ // Ditto for the bytes after the put
+ var bytesAfterPut: ByteBuffer = null
+
+ // Size of the block in bytes (to return to caller)
+ var size = 0L
+
locker.getLock(blockId).synchronized {
logDebug("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
+ " to get into synchronized block")
-
- // Check and warn if block with same id already exists
- if (getLevel(blockId) != null) {
- logWarning("Block " + blockId + " already exists in local machine")
- return
- }
- if (level.useMemory && level.useDisk) {
- // If saving to both memory and disk, then serialize only once
- memoryStore.putValues(blockId, values, level) match {
- case Left(newValues) =>
- diskStore.putValues(blockId, newValues, level) match {
- case Right(newBytes) => bytes = newBytes
- case _ => throw new Exception("Unexpected return value")
- }
- case Right(newBytes) =>
- bytes = newBytes
- diskStore.putBytes(blockId, newBytes, level)
- }
- } else if (level.useMemory) {
- // If only save to memory
- memoryStore.putValues(blockId, values, level) match {
- case Right(newBytes) => bytes = newBytes
+ if (level.useMemory) {
+ // Save it just to memory first, even if it also has useDisk set to true; we will later
+ // drop it to disk if the memory store can't hold it.
+ val res = memoryStore.putValues(blockId, values, level, true)
+ size = res.size
+ res.data match {
+ case Right(newBytes) => bytesAfterPut = newBytes
case Left(newIterator) => valuesAfterPut = newIterator
}
} else {
- // If only save to disk
- diskStore.putValues(blockId, values, level) match {
- case Right(newBytes) => bytes = newBytes
- case _ => throw new Exception("Unexpected return value")
+ // Save directly to disk.
+ val askForBytes = level.replication > 1 // Don't get back the bytes unless we replicate them
+ val res = diskStore.putValues(blockId, values, level, askForBytes)
+ size = res.size
+ res.data match {
+ case Right(newBytes) => bytesAfterPut = newBytes
+ case _ =>
}
}
-
- // Store the storage level
- setLevel(blockId, level, tellMaster)
+
+ // Now that the block is in either the memory or disk store, let other threads read it,
+ // and tell the master about it.
+ myInfo.markReady(size)
+ if (tellMaster) {
+ reportBlockStatus(blockId)
+ }
}
logDebug("Put block " + blockId + " locally took " + Utils.getUsedTimeMs(startTimeMs))
- // Replicate block if required
+ // Replicate block if required
if (level.replication > 1) {
// Serialize the block if not already done
- if (bytes == null) {
+ if (bytesAfterPut == null) {
if (valuesAfterPut == null) {
throw new SparkException(
"Underlying put returned neither an Iterator nor bytes! This shouldn't happen.")
}
- bytes = dataSerialize(valuesAfterPut)
+ bytesAfterPut = dataSerialize(blockId, valuesAfterPut)
}
- replicate(blockId, bytes, level)
+ replicate(blockId, bytesAfterPut, level)
}
+ BlockManager.dispose(bytesAfterPut)
+
// TODO: This code will be removed when CacheTracker is gone.
if (blockId.startsWith("rdd")) {
- notifyTheCacheTracker(blockId)
+ notifyCacheTracker(blockId)
}
logDebug("Put block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs))
+
+ return size
}
/**
* Put a new block of serialized bytes to the block manager.
*/
- def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) {
+ def putBytes(
+ blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) {
+
if (blockId == null) {
throw new IllegalArgumentException("Block Id is null")
}
@@ -449,10 +656,21 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
if (level == null || !level.isValid) {
throw new IllegalArgumentException("Storage level is null or invalid")
}
-
- val startTimeMs = System.currentTimeMillis
-
- // Initiate the replication before storing it locally. This is faster as
+
+ if (blockInfo.containsKey(blockId)) {
+ logWarning("Block " + blockId + " already exists on this machine; not re-adding it")
+ return
+ }
+
+ // Remember the block's storage level so that we can correctly drop it to disk if it needs
+ // to be dropped right after it got put into memory. Note, however, that other threads will
+ // not be able to get() this block until we call markReady on its BlockInfo.
+ val myInfo = new BlockInfo(level, tellMaster)
+ blockInfo.put(blockId, myInfo)
+
+ val startTimeMs = System.currentTimeMillis
+
+ // Initiate the replication before storing it locally. This is faster as
// data is already serialized and ready for sending
val replicationFuture = if (level.replication > 1) {
val bufferView = bytes.duplicate() // Doesn't copy the bytes, just creates a wrapper
@@ -466,27 +684,29 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
locker.getLock(blockId).synchronized {
logDebug("PutBytes for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs)
+ " to get into synchronized block")
- if (getLevel(blockId) != null) {
- logWarning("Block " + blockId + " already exists")
- return
- }
if (level.useMemory) {
+ // Store it only in memory at first, even if useDisk is also set to true
+ bytes.rewind()
memoryStore.putBytes(blockId, bytes, level)
- }
- if (level.useDisk) {
+ } else {
+ bytes.rewind()
diskStore.putBytes(blockId, bytes, level)
}
- // Store the storage level
- setLevel(blockId, level, tellMaster)
+ // Now that the block is in either the memory or disk store, let other threads read it,
+ // and tell the master about it.
+ myInfo.markReady(bytes.limit)
+ if (tellMaster) {
+ reportBlockStatus(blockId)
+ }
}
// TODO: This code will be removed when CacheTracker is gone.
if (blockId.startsWith("rdd")) {
- notifyTheCacheTracker(blockId)
+ notifyCacheTracker(blockId)
}
-
+
// If replication had started, then wait for it to finish
if (level.replication > 1) {
if (replicationFuture == null) {
@@ -495,12 +715,11 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
Await.ready(replicationFuture, Duration.Inf)
}
- val finishTime = System.currentTimeMillis
if (level.replication > 1) {
- logDebug("PutBytes for block " + blockId + " with replication took " +
+ logDebug("PutBytes for block " + blockId + " with replication took " +
Utils.getUsedTimeMs(startTimeMs))
} else {
- logDebug("PutBytes for block " + blockId + " without replication took " +
+ logDebug("PutBytes for block " + blockId + " without replication took " +
Utils.getUsedTimeMs(startTimeMs))
}
}
@@ -508,17 +727,14 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
/**
* Replicate block to another node.
*/
-
- var firstTime = true
- var peers : Seq[BlockManagerId] = null
+ var cachedPeers: Seq[BlockManagerId] = null
private def replicate(blockId: String, data: ByteBuffer, level: StorageLevel) {
val tLevel: StorageLevel =
new StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1)
- if (firstTime) {
- peers = master.mustGetPeers(GetPeers(blockManagerId, level.replication - 1))
- firstTime = false;
- }
- for (peer: BlockManagerId <- peers) {
+ if (cachedPeers == null) {
+ cachedPeers = master.mustGetPeers(GetPeers(blockManagerId, level.replication - 1))
+ }
+ for (peer: BlockManagerId <- cachedPeers) {
val start = System.nanoTime
data.rewind()
logDebug("Try to replicate BlockId " + blockId + " once; The size of the data is "
@@ -534,19 +750,20 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
}
// TODO: This code will be removed when CacheTracker is gone.
- private def notifyTheCacheTracker(key: String) {
- val rddInfo = key.split(":")
- val rddId: Int = rddInfo(1).toInt
- val splitIndex: Int = rddInfo(2).toInt
- val host = System.getProperty("spark.hostname", Utils.localHostName)
- cacheTracker.notifyTheCacheTrackerFromBlockManager(spark.AddedToCache(rddId, splitIndex, host))
+ private def notifyCacheTracker(key: String) {
+ if (cacheTracker != null) {
+ val rddInfo = key.split("_")
+ val rddId: Int = rddInfo(1).toInt
+ val partition: Int = rddInfo(2).toInt
+ cacheTracker.notifyFromBlockManager(spark.AddedToCache(rddId, partition, host))
+ }
}
/**
* Read a block consisting of a single object.
*/
def getSingle(blockId: String): Option[Any] = {
- get(blockId).map(_.next)
+ get(blockId).map(_.next())
}
/**
@@ -557,42 +774,76 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
}
/**
- * Drop block from memory (called when memory store has reached it limit)
+ * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory
+ * store reaches its limit and needs to free up space.
*/
- def dropFromMemory(blockId: String) {
+ def dropFromMemory(blockId: String, data: Either[ArrayBuffer[Any], ByteBuffer]) {
+ logInfo("Dropping block " + blockId + " from memory")
locker.getLock(blockId).synchronized {
- val level = getLevel(blockId)
- if (level == null) {
- logWarning("Block " + blockId + " cannot be removed from memory as it does not exist")
- return
+ val info = blockInfo.get(blockId)
+ val level = info.level
+ if (level.useDisk && !diskStore.contains(blockId)) {
+ logInfo("Writing block " + blockId + " to disk")
+ data match {
+ case Left(elements) =>
+ diskStore.putValues(blockId, elements, level, false)
+ case Right(bytes) =>
+ diskStore.putBytes(blockId, bytes, level)
+ }
+ }
+ memoryStore.remove(blockId)
+ if (info.tellMaster) {
+ reportBlockStatus(blockId)
}
- if (!level.useMemory) {
- logWarning("Block " + blockId + " cannot be removed from memory as it is not in memory")
- return
+ if (!level.useDisk) {
+ // The block is completely gone from this node; forget it so we can put() it again later.
+ blockInfo.remove(blockId)
}
- memoryStore.remove(blockId)
- val newLevel = new StorageLevel(level.useDisk, false, level.deserialized, level.replication)
- setLevel(blockId, newLevel)
}
}
- def dataSerialize(values: Iterator[Any]): ByteBuffer = {
- /*serializer.newInstance().serializeMany(values)*/
+ def shouldCompress(blockId: String): Boolean = {
+ if (blockId.startsWith("shuffle_")) {
+ compressShuffle
+ } else if (blockId.startsWith("broadcast_")) {
+ compressBroadcast
+ } else if (blockId.startsWith("rdd_")) {
+ compressRdds
+ } else {
+ false // Won't happen in a real cluster, but it can in tests
+ }
+ }
+
+ /**
+ * Wrap an output stream for compression if block compression is enabled for its block type
+ */
+ def wrapForCompression(blockId: String, s: OutputStream): OutputStream = {
+ if (shouldCompress(blockId)) new LZFOutputStream(s) else s
+ }
+
+ /**
+ * Wrap an input stream for compression if block compression is enabled for its block type
+ */
+ def wrapForCompression(blockId: String, s: InputStream): InputStream = {
+ if (shouldCompress(blockId)) new LZFInputStream(s) else s
+ }
+
+ def dataSerialize(blockId: String, values: Iterator[Any]): ByteBuffer = {
val byteStream = new FastByteArrayOutputStream(4096)
- serializer.newInstance().serializeStream(byteStream).writeAll(values).close()
+ val ser = serializer.newInstance()
+ ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close()
byteStream.trim()
ByteBuffer.wrap(byteStream.array)
}
- def dataDeserialize(bytes: ByteBuffer): Iterator[Any] = {
- /*serializer.newInstance().deserializeMany(bytes)*/
- val ser = serializer.newInstance()
+ /**
+ * Deserializes a ByteBuffer into an iterator of values and disposes of it when the end of
+ * the iterator is reached.
+ */
+ def dataDeserialize(blockId: String, bytes: ByteBuffer): Iterator[Any] = {
bytes.rewind()
- return ser.deserializeStream(new ByteBufferInputStream(bytes)).toIterator
- }
-
- private def notifyMaster(heartBeat: HeartBeat) {
- master.mustHeartBeat(heartBeat)
+ val stream = wrapForCompression(blockId, new ByteBufferInputStream(bytes, true))
+ serializer.newInstance().deserializeStream(stream).asIterator
}
def stop() {
@@ -604,9 +855,25 @@ class BlockManager(val master: BlockManagerMaster, val serializer: Serializer, m
}
}
-object BlockManager {
- def getMaxMemoryFromSystemProperties(): Long = {
+private[spark]
+object BlockManager extends Logging {
+ def getMaxMemoryFromSystemProperties: Long = {
val memoryFraction = System.getProperty("spark.storage.memoryFraction", "0.66").toDouble
(Runtime.getRuntime.maxMemory * memoryFraction).toLong
}
+
+ /**
+ * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that
+ * might cause errors if one attempts to read from the unmapped buffer, but it's better than
+ * waiting for the GC to find it because that could lead to huge numbers of open files. There's
+ * unfortunately no standard API to do this.
+ */
+ def dispose(buffer: ByteBuffer) {
+ if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) {
+ logDebug("Unmapping " + buffer)
+ if (buffer.asInstanceOf[DirectBuffer].cleaner() != null) {
+ buffer.asInstanceOf[DirectBuffer].cleaner().clean()
+ }
+ }
+ }
}
diff --git a/core/src/main/scala/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
index 9f03c5a32c..b3345623b3 100644
--- a/core/src/main/scala/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerMaster.scala
@@ -3,37 +3,35 @@ package spark.storage
import java.io._
import java.util.{HashMap => JHashMap}
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import scala.util.Random
import akka.actor._
import akka.dispatch._
import akka.pattern.ask
import akka.remote._
-import akka.util.Duration
-import akka.util.Timeout
+import akka.util.{Duration, Timeout}
import akka.util.duration._
-import spark.Logging
-import spark.SparkException
-import spark.Utils
+import spark.{Logging, SparkException, Utils}
+
+private[spark]
sealed trait ToBlockManagerMaster
+private[spark]
case class RegisterBlockManager(
blockManagerId: BlockManagerId,
- maxMemSize: Long,
- maxDiskSize: Long)
+ maxMemSize: Long)
extends ToBlockManagerMaster
-
+
+private[spark]
class HeartBeat(
var blockManagerId: BlockManagerId,
var blockId: String,
var storageLevel: StorageLevel,
- var deserializedSize: Long,
- var size: Long)
+ var memSize: Long,
+ var diskSize: Long)
extends ToBlockManagerMaster
with Externalizable {
@@ -43,8 +41,8 @@ class HeartBeat(
blockManagerId.writeExternal(out)
out.writeUTF(blockId)
storageLevel.writeExternal(out)
- out.writeInt(deserializedSize.toInt)
- out.writeInt(size.toInt)
+ out.writeInt(memSize.toInt)
+ out.writeInt(diskSize.toInt)
}
override def readExternal(in: ObjectInput) {
@@ -53,105 +51,115 @@ class HeartBeat(
blockId = in.readUTF()
storageLevel = new StorageLevel()
storageLevel.readExternal(in)
- deserializedSize = in.readInt()
- size = in.readInt()
+ memSize = in.readInt()
+ diskSize = in.readInt()
}
}
+private[spark]
object HeartBeat {
def apply(blockManagerId: BlockManagerId,
blockId: String,
storageLevel: StorageLevel,
- deserializedSize: Long,
- size: Long): HeartBeat = {
- new HeartBeat(blockManagerId, blockId, storageLevel, deserializedSize, size)
+ memSize: Long,
+ diskSize: Long): HeartBeat = {
+ new HeartBeat(blockManagerId, blockId, storageLevel, memSize, diskSize)
}
-
// For pattern-matching
def unapply(h: HeartBeat): Option[(BlockManagerId, String, StorageLevel, Long, Long)] = {
- Some((h.blockManagerId, h.blockId, h.storageLevel, h.deserializedSize, h.size))
+ Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize))
}
}
-
+
+private[spark]
case class GetLocations(blockId: String) extends ToBlockManagerMaster
+private[spark]
case class GetLocationsMultipleBlockIds(blockIds: Array[String]) extends ToBlockManagerMaster
-
+
+private[spark]
case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster
-
+
+private[spark]
case class RemoveHost(host: String) extends ToBlockManagerMaster
+private[spark]
case object StopBlockManagerMaster extends ToBlockManagerMaster
+private[spark]
+case object GetMemoryStatus extends ToBlockManagerMaster
+
+
+private[spark] class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
-class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
-
class BlockManagerInfo(
+ val blockManagerId: BlockManagerId,
timeMs: Long,
- maxMem: Long,
- maxDisk: Long) {
- private var lastSeenMs = timeMs
- private var remainedMem = maxMem
- private var remainedDisk = maxDisk
- private val blocks = new JHashMap[String, StorageLevel]
-
+ val maxMem: Long) {
+ private var _lastSeenMs = timeMs
+ private var _remainingMem = maxMem
+ private val _blocks = new JHashMap[String, StorageLevel]
+
+ logInfo("Registering block manager %s:%d with %s RAM".format(
+ blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(maxMem)))
+
def updateLastSeenMs() {
- lastSeenMs = System.currentTimeMillis() / 1000
+ _lastSeenMs = System.currentTimeMillis() / 1000
}
-
- def addBlock(blockId: String, storageLevel: StorageLevel, deserializedSize: Long, size: Long) =
- synchronized {
+
+ def updateBlockInfo(blockId: String, storageLevel: StorageLevel, memSize: Long, diskSize: Long)
+ : Unit = synchronized {
+
updateLastSeenMs()
-
- if (blocks.containsKey(blockId)) {
- val oriLevel: StorageLevel = blocks.get(blockId)
-
- if (oriLevel.deserialized) {
- remainedMem += deserializedSize
- }
- if (oriLevel.useMemory) {
- remainedMem += size
- }
- if (oriLevel.useDisk) {
- remainedDisk += size
+
+ if (_blocks.containsKey(blockId)) {
+ // The block exists on the slave already.
+ val originalLevel: StorageLevel = _blocks.get(blockId)
+
+ if (originalLevel.useMemory) {
+ _remainingMem += memSize
}
}
-
- if (storageLevel.isValid) {
- blocks.put(blockId, storageLevel)
- if (storageLevel.deserialized) {
- remainedMem -= deserializedSize
- }
+
+ if (storageLevel.isValid) {
+ // isValid means it is either stored in-memory or on-disk.
+ _blocks.put(blockId, storageLevel)
if (storageLevel.useMemory) {
- remainedMem -= size
+ _remainingMem -= memSize
+ logInfo("Added %s in memory on %s:%d (size: %s, free: %s)".format(
+ blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize),
+ Utils.memoryBytesToString(_remainingMem)))
}
if (storageLevel.useDisk) {
- remainedDisk -= size
+ logInfo("Added %s on disk on %s:%d (size: %s)".format(
+ blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize)))
+ }
+ } else if (_blocks.containsKey(blockId)) {
+ // If isValid is not true, drop the block.
+ val originalLevel: StorageLevel = _blocks.get(blockId)
+ _blocks.remove(blockId)
+ if (originalLevel.useMemory) {
+ _remainingMem += memSize
+ logInfo("Removed %s on %s:%d in memory (size: %s, free: %s)".format(
+ blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(memSize),
+ Utils.memoryBytesToString(_remainingMem)))
+ }
+ if (originalLevel.useDisk) {
+ logInfo("Removed %s on %s:%d on disk (size: %s)".format(
+ blockId, blockManagerId.ip, blockManagerId.port, Utils.memoryBytesToString(diskSize)))
}
- } else {
- blocks.remove(blockId)
}
}
- def getLastSeenMs: Long = {
- return lastSeenMs
- }
-
- def getRemainedMem: Long = {
- return remainedMem
- }
-
- def getRemainedDisk: Long = {
- return remainedDisk
- }
+ def remainingMem: Long = _remainingMem
- override def toString: String = {
- return "BlockManagerInfo " + timeMs + " " + remainedMem + " " + remainedDisk
- }
+ def lastSeenMs: Long = _lastSeenMs
+
+ override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem
def clear() {
- blocks.clear()
+ _blocks.clear()
}
}
@@ -159,7 +167,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
private val blockInfo = new JHashMap[String, Pair[Int, HashSet[BlockManagerId]]]
initLogging()
-
+
def removeHost(host: String) {
logInfo("Trying to remove the host: " + host + " from BlockManagerMaster.")
logInfo("Previous hosts: " + blockManagerInfo.keySet.toSeq)
@@ -171,8 +179,8 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
}
def receive = {
- case RegisterBlockManager(blockManagerId, maxMemSize, maxDiskSize) =>
- register(blockManagerId, maxMemSize, maxDiskSize)
+ case RegisterBlockManager(blockManagerId, maxMemSize) =>
+ register(blockManagerId, maxMemSize)
case HeartBeat(blockManagerId, blockId, storageLevel, deserializedSize, size) =>
heartBeat(blockManagerId, blockId, storageLevel, deserializedSize, size)
@@ -186,7 +194,10 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
case GetPeers(blockManagerId, size) =>
getPeersDeterministic(blockManagerId, size)
/*getPeers(blockManagerId, size)*/
-
+
+ case GetMemoryStatus =>
+ getMemoryStatus
+
case RemoveHost(host) =>
removeHost(host)
sender ! true
@@ -196,43 +207,50 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
sender ! true
context.stop(self)
- case other =>
+ case other =>
logInfo("Got unknown message: " + other)
}
-
- private def register(blockManagerId: BlockManagerId, maxMemSize: Long, maxDiskSize: Long) {
+
+ // Return a map from the block manager id to max memory and remaining memory.
+ private def getMemoryStatus() {
+ val res = blockManagerInfo.map { case(blockManagerId, info) =>
+ (blockManagerId, (info.maxMem, info.remainingMem))
+ }.toMap
+ sender ! res
+ }
+
+ private def register(blockManagerId: BlockManagerId, maxMemSize: Long) {
val startTimeMs = System.currentTimeMillis()
val tmp = " " + blockManagerId + " "
logDebug("Got in register 0" + tmp + Utils.getUsedTimeMs(startTimeMs))
- logInfo("Got Register Msg from " + blockManagerId)
if (blockManagerId.ip == Utils.localHostName() && !isLocal) {
logInfo("Got Register Msg from master node, don't register it")
} else {
blockManagerInfo += (blockManagerId -> new BlockManagerInfo(
- System.currentTimeMillis() / 1000, maxMemSize, maxDiskSize))
+ blockManagerId, System.currentTimeMillis() / 1000, maxMemSize))
}
logDebug("Got in register 1" + tmp + Utils.getUsedTimeMs(startTimeMs))
sender ! true
}
-
+
private def heartBeat(
blockManagerId: BlockManagerId,
blockId: String,
storageLevel: StorageLevel,
- deserializedSize: Long,
- size: Long) {
-
+ memSize: Long,
+ diskSize: Long) {
+
val startTimeMs = System.currentTimeMillis()
val tmp = " " + blockManagerId + " " + blockId + " "
-
+
if (blockId == null) {
blockManagerInfo(blockManagerId).updateLastSeenMs()
logDebug("Got in heartBeat 1" + tmp + " used " + Utils.getUsedTimeMs(startTimeMs))
sender ! true
}
-
- blockManagerInfo(blockManagerId).addBlock(blockId, storageLevel, deserializedSize, size)
-
+
+ blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize)
+
var locations: HashSet[BlockManagerId] = null
if (blockInfo.containsKey(blockId)) {
locations = blockInfo.get(blockId)._2
@@ -240,19 +258,19 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
locations = new HashSet[BlockManagerId]
blockInfo.put(blockId, (storageLevel.replication, locations))
}
-
+
if (storageLevel.isValid) {
locations += blockManagerId
} else {
locations.remove(blockManagerId)
}
-
+
if (locations.size == 0) {
blockInfo.remove(blockId)
}
sender ! true
}
-
+
private def getLocations(blockId: String) {
val startTimeMs = System.currentTimeMillis()
val tmp = " " + blockId + " "
@@ -260,7 +278,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
if (blockInfo.containsKey(blockId)) {
var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId]
res.appendAll(blockInfo.get(blockId)._2)
- logDebug("Got in getLocations 1" + tmp + " as "+ res.toSeq + " at "
+ logDebug("Got in getLocations 1" + tmp + " as "+ res.toSeq + " at "
+ Utils.getUsedTimeMs(startTimeMs))
sender ! res.toSeq
} else {
@@ -269,7 +287,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
sender ! res
}
}
-
+
private def getLocationsMultipleBlockIds(blockIds: Array[String]) {
def getLocations(blockId: String): Seq[BlockManagerId] = {
val tmp = blockId
@@ -285,7 +303,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
return res.toSeq
}
}
-
+
logDebug("Got in getLocationsMultipleBlockIds " + blockIds.toSeq)
var res: ArrayBuffer[Seq[BlockManagerId]] = new ArrayBuffer[Seq[BlockManagerId]]
for (blockId <- blockIds) {
@@ -306,7 +324,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
}
sender ! res.toSeq
}
-
+
private def getPeersDeterministic(blockManagerId: BlockManagerId, size: Int) {
var peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray
var res: ArrayBuffer[BlockManagerId] = new ArrayBuffer[BlockManagerId]
@@ -329,7 +347,7 @@ class BlockManagerMasterActor(val isLocal: Boolean) extends Actor with Logging {
}
}
-class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: Boolean)
+private[spark] class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: Boolean)
extends Logging {
val AKKA_ACTOR_NAME: String = "BlockMasterManager"
@@ -352,7 +370,7 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
logInfo("Connecting to BlockManagerMaster: " + url)
masterActor = actorSystem.actorFor(url)
}
-
+
def stop() {
if (masterActor != null) {
communicate(StopBlockManagerMaster)
@@ -379,17 +397,19 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
throw new SparkException("Error reply received from BlockManagerMaster")
}
}
-
+
def notifyADeadHost(host: String) {
communicate(RemoveHost(host + ":" + DEFAULT_MANAGER_PORT))
logInfo("Removed " + host + " successfully in notifyADeadHost")
}
def mustRegisterBlockManager(msg: RegisterBlockManager) {
+ logInfo("Trying to register BlockManager")
while (! syncRegisterBlockManager(msg)) {
logWarning("Failed to register " + msg)
Thread.sleep(REQUEST_RETRY_INTERVAL_MS)
}
+ logInfo("Done registering BlockManager")
}
def syncRegisterBlockManager(msg: RegisterBlockManager): Boolean = {
@@ -397,7 +417,7 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
val startTimeMs = System.currentTimeMillis()
val tmp = " msg " + msg + " "
logDebug("Got in syncRegisterBlockManager 0 " + tmp + Utils.getUsedTimeMs(startTimeMs))
-
+
try {
communicate(msg)
logInfo("BlockManager registered successfully @ syncRegisterBlockManager")
@@ -409,19 +429,19 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
return false
}
}
-
+
def mustHeartBeat(msg: HeartBeat) {
while (! syncHeartBeat(msg)) {
logWarning("Failed to send heartbeat" + msg)
Thread.sleep(REQUEST_RETRY_INTERVAL_MS)
}
}
-
+
def syncHeartBeat(msg: HeartBeat): Boolean = {
val startTimeMs = System.currentTimeMillis()
val tmp = " msg " + msg + " "
logDebug("Got in syncHeartBeat " + tmp + " 0 " + Utils.getUsedTimeMs(startTimeMs))
-
+
try {
communicate(msg)
logDebug("Heartbeat sent successfully")
@@ -433,7 +453,7 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
return false
}
}
-
+
def mustGetLocations(msg: GetLocations): Seq[BlockManagerId] = {
var res = syncGetLocations(msg)
while (res == null) {
@@ -443,7 +463,7 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
}
return res
}
-
+
def syncGetLocations(msg: GetLocations): Seq[BlockManagerId] = {
val startTimeMs = System.currentTimeMillis()
val tmp = " msg " + msg + " "
@@ -476,13 +496,13 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
}
return res
}
-
+
def syncGetLocationsMultipleBlockIds(msg: GetLocationsMultipleBlockIds):
Seq[Seq[BlockManagerId]] = {
val startTimeMs = System.currentTimeMillis
val tmp = " msg " + msg + " "
logDebug("Got in syncGetLocationsMultipleBlockIds 0 " + tmp + Utils.getUsedTimeMs(startTimeMs))
-
+
try {
val answer = askMaster(msg).asInstanceOf[Seq[Seq[BlockManagerId]]]
if (answer != null) {
@@ -500,7 +520,7 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
return null
}
}
-
+
def mustGetPeers(msg: GetPeers): Seq[BlockManagerId] = {
var res = syncGetPeers(msg)
while ((res == null) || (res.length != msg.size)) {
@@ -508,10 +528,10 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
Thread.sleep(REQUEST_RETRY_INTERVAL_MS)
res = syncGetPeers(msg)
}
-
+
return res
}
-
+
def syncGetPeers(msg: GetPeers): Seq[BlockManagerId] = {
val startTimeMs = System.currentTimeMillis
val tmp = " msg " + msg + " "
@@ -533,4 +553,8 @@ class BlockManagerMaster(actorSystem: ActorSystem, isMaster: Boolean, isLocal: B
return null
}
}
+
+ def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = {
+ askMaster(GetMemoryStatus).asInstanceOf[Map[BlockManagerId, (Long, Long)]]
+ }
}
diff --git a/core/src/main/scala/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/spark/storage/BlockManagerWorker.scala
index 0658a57187..d2985559c1 100644
--- a/core/src/main/scala/spark/storage/BlockManagerWorker.scala
+++ b/core/src/main/scala/spark/storage/BlockManagerWorker.scala
@@ -1,25 +1,23 @@
package spark.storage
-import java.nio._
+import java.nio.ByteBuffer
import scala.actors._
import scala.actors.Actor._
import scala.actors.remote._
-
-import scala.collection.mutable.ArrayBuffer
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.HashSet
+import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import scala.util.Random
-import spark.Logging
-import spark.Utils
-import spark.SparkEnv
+import spark.{Logging, Utils, SparkEnv}
import spark.network._
/**
- * This should be changed to use event model late.
+ * A network interface for BlockManager. Each slave should have one
+ * BlockManagerWorker.
+ *
+ * TODO: Use event model.
*/
-class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
+private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
initLogging()
blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive)
@@ -32,11 +30,10 @@ class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
logDebug("Handling as a buffer message " + bufferMessage)
val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage)
logDebug("Parsed as a block message array")
- val responseMessages = blockMessages.map(processBlockMessage _).filter(_ != None).map(_.get)
- /*logDebug("Processed block messages")*/
+ val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get)
return Some(new BlockMessageArray(responseMessages).toBufferMessage)
} catch {
- case e: Exception => logError("Exception handling buffer message: " + e.getMessage)
+ case e: Exception => logError("Exception handling buffer message", e)
return None
}
}
@@ -51,13 +48,13 @@ class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
blockMessage.getType match {
case BlockMessage.TYPE_PUT_BLOCK => {
val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel)
- logInfo("Received [" + pB + "]")
+ logDebug("Received [" + pB + "]")
putBlock(pB.id, pB.data, pB.level)
return None
}
case BlockMessage.TYPE_GET_BLOCK => {
val gB = new GetBlock(blockMessage.getId)
- logInfo("Received [" + gB + "]")
+ logDebug("Received [" + gB + "]")
val buffer = getBlock(gB.id)
if (buffer == null) {
return None
@@ -78,17 +75,10 @@ class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
private def getBlock(id: String): ByteBuffer = {
val startTimeMs = System.currentTimeMillis()
- logDebug("Getblock " + id + " started from " + startTimeMs)
- val block = blockManager.getLocal(id)
- val buffer = block match {
- case Some(tValues) => {
- val values = tValues
- val buffer = blockManager.dataSerialize(values)
- buffer
- }
- case None => {
- null
- }
+ logDebug("GetBlock " + id + " started from " + startTimeMs)
+ val buffer = blockManager.getLocalBytes(id) match {
+ case Some(bytes) => bytes
+ case None => null
}
logDebug("GetBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs)
+ " and got buffer " + buffer)
@@ -96,7 +86,7 @@ class BlockManagerWorker(val blockManager: BlockManager) extends Logging {
}
}
-object BlockManagerWorker extends Logging {
+private[spark] object BlockManagerWorker extends Logging {
private var blockManagerWorker: BlockManagerWorker = null
private val DATA_TRANSFER_TIME_OUT_MS: Long = 500
private val REQUEST_RETRY_INTERVAL_MS: Long = 1000
diff --git a/core/src/main/scala/spark/storage/BlockMessage.scala b/core/src/main/scala/spark/storage/BlockMessage.scala
index 607633c6df..3f234df654 100644
--- a/core/src/main/scala/spark/storage/BlockMessage.scala
+++ b/core/src/main/scala/spark/storage/BlockMessage.scala
@@ -1,6 +1,6 @@
package spark.storage
-import java.nio._
+import java.nio.ByteBuffer
import scala.collection.mutable.StringBuilder
import scala.collection.mutable.ArrayBuffer
@@ -8,11 +8,11 @@ import scala.collection.mutable.ArrayBuffer
import spark._
import spark.network._
-case class GetBlock(id: String)
-case class GotBlock(id: String, data: ByteBuffer)
-case class PutBlock(id: String, data: ByteBuffer, level: StorageLevel)
+private[spark] case class GetBlock(id: String)
+private[spark] case class GotBlock(id: String, data: ByteBuffer)
+private[spark] case class PutBlock(id: String, data: ByteBuffer, level: StorageLevel)
-class BlockMessage() {
+private[spark] class BlockMessage() {
// Un-initialized: typ = 0
// GetBlock: typ = 1
// GotBlock: typ = 2
@@ -158,7 +158,7 @@ class BlockMessage() {
}
}
-object BlockMessage {
+private[spark] object BlockMessage {
val TYPE_NON_INITIALIZED: Int = 0
val TYPE_GET_BLOCK: Int = 1
val TYPE_GOT_BLOCK: Int = 2
@@ -196,7 +196,7 @@ object BlockMessage {
def main(args: Array[String]) {
val B = new BlockMessage()
- B.set(new PutBlock("ABC", ByteBuffer.allocate(10), StorageLevel.DISK_AND_MEMORY_2))
+ B.set(new PutBlock("ABC", ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2))
val bMsg = B.toBufferMessage
val C = new BlockMessage()
C.set(bMsg)
diff --git a/core/src/main/scala/spark/storage/BlockMessageArray.scala b/core/src/main/scala/spark/storage/BlockMessageArray.scala
index 497a19856e..a25decb123 100644
--- a/core/src/main/scala/spark/storage/BlockMessageArray.scala
+++ b/core/src/main/scala/spark/storage/BlockMessageArray.scala
@@ -1,5 +1,6 @@
package spark.storage
-import java.nio._
+
+import java.nio.ByteBuffer
import scala.collection.mutable.StringBuilder
import scala.collection.mutable.ArrayBuffer
@@ -7,6 +8,7 @@ import scala.collection.mutable.ArrayBuffer
import spark._
import spark.network._
+private[spark]
class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging {
def this(bm: BlockMessage) = this(Array(bm))
@@ -84,7 +86,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockM
}
}
-object BlockMessageArray {
+private[spark] object BlockMessageArray {
def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = {
val newBlockMessageArray = new BlockMessageArray()
@@ -98,7 +100,7 @@ object BlockMessageArray {
if (i % 2 == 0) {
val buffer = ByteBuffer.allocate(100)
buffer.clear
- BlockMessage.fromPutBlock(PutBlock(i.toString, buffer, StorageLevel.MEMORY_ONLY))
+ BlockMessage.fromPutBlock(PutBlock(i.toString, buffer, StorageLevel.MEMORY_ONLY_SER))
} else {
BlockMessage.fromGetBlock(GetBlock(i.toString))
}
diff --git a/core/src/main/scala/spark/storage/BlockStore.scala b/core/src/main/scala/spark/storage/BlockStore.scala
index 77e0ed84c5..096bf8bdd9 100644
--- a/core/src/main/scala/spark/storage/BlockStore.scala
+++ b/core/src/main/scala/spark/storage/BlockStore.scala
@@ -1,25 +1,31 @@
package spark.storage
-import spark.{Utils, Logging, Serializer, SizeEstimator}
-import scala.collection.mutable.ArrayBuffer
-import java.io.{File, RandomAccessFile}
import java.nio.ByteBuffer
-import java.nio.channels.FileChannel.MapMode
-import java.util.{UUID, LinkedHashMap}
-import java.util.concurrent.Executors
-import java.util.concurrent.ConcurrentHashMap
-import it.unimi.dsi.fastutil.io._
-import java.util.concurrent.ArrayBlockingQueue
+import scala.collection.mutable.ArrayBuffer
+
+import spark.Logging
/**
* Abstract class to store blocks
*/
-abstract class BlockStore(blockManager: BlockManager) extends Logging {
- initLogging()
-
- def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel)
-
- def putValues(blockId: String, values: Iterator[Any], level: StorageLevel): Either[Iterator[Any], ByteBuffer]
+private[spark]
+abstract class BlockStore(val blockManager: BlockManager) extends Logging {
+ def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel)
+
+ /**
+ * Put in a block and, possibly, also return its content as either bytes or another Iterator.
+ * This is used to efficiently write the values to multiple locations (e.g. for replication).
+ *
+ * @return a PutResult that contains the size of the data, as well as the values put if
+ * returnValues is true (if not, the result's data field can be null)
+ */
+ def putValues(blockId: String, values: ArrayBuffer[Any], level: StorageLevel,
+ returnValues: Boolean) : PutResult
+
+ /**
+ * Return the size of a block in bytes.
+ */
+ def getSize(blockId: String): Long
def getBytes(blockId: String): Option[ByteBuffer]
@@ -27,282 +33,7 @@ abstract class BlockStore(blockManager: BlockManager) extends Logging {
def remove(blockId: String)
- def dataSerialize(values: Iterator[Any]): ByteBuffer = blockManager.dataSerialize(values)
-
- def dataDeserialize(bytes: ByteBuffer): Iterator[Any] = blockManager.dataDeserialize(bytes)
+ def contains(blockId: String): Boolean
def clear() { }
}
-
-/**
- * Class to store blocks in memory
- */
-class MemoryStore(blockManager: BlockManager, maxMemory: Long)
- extends BlockStore(blockManager) {
-
- case class Entry(value: Any, size: Long, deserialized: Boolean, var dropPending: Boolean = false)
-
- private val memoryStore = new LinkedHashMap[String, Entry](32, 0.75f, true)
- private var currentMemory = 0L
-
- //private val blockDropper = Executors.newSingleThreadExecutor()
- private val blocksToDrop = new ArrayBlockingQueue[String](10000, true)
- private val blockDropper = new Thread("memory store - block dropper") {
- override def run() {
- try{
- while (true) {
- val blockId = blocksToDrop.take()
- logDebug("Block " + blockId + " ready to be dropped")
- blockManager.dropFromMemory(blockId)
- }
- } catch {
- case ie: InterruptedException =>
- logInfo("Shutting down block dropper")
- }
- }
- }
- blockDropper.start()
-
- def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
- if (level.deserialized) {
- bytes.rewind()
- val values = dataDeserialize(bytes)
- val elements = new ArrayBuffer[Any]
- elements ++= values
- val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef])
- ensureFreeSpace(sizeEstimate)
- val entry = new Entry(elements, sizeEstimate, true)
- memoryStore.synchronized { memoryStore.put(blockId, entry) }
- currentMemory += sizeEstimate
- logDebug("Block " + blockId + " stored as values to memory")
- } else {
- val entry = new Entry(bytes, bytes.limit, false)
- ensureFreeSpace(bytes.limit)
- memoryStore.synchronized { memoryStore.put(blockId, entry) }
- currentMemory += bytes.limit
- logDebug("Block " + blockId + " stored as " + bytes.limit + " bytes to memory")
- }
- }
-
- def putValues(blockId: String, values: Iterator[Any], level: StorageLevel): Either[Iterator[Any], ByteBuffer] = {
- if (level.deserialized) {
- val elements = new ArrayBuffer[Any]
- elements ++= values
- val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef])
- ensureFreeSpace(sizeEstimate)
- val entry = new Entry(elements, sizeEstimate, true)
- memoryStore.synchronized { memoryStore.put(blockId, entry) }
- currentMemory += sizeEstimate
- logDebug("Block " + blockId + " stored as values to memory")
- return Left(elements.iterator)
- } else {
- val bytes = dataSerialize(values)
- ensureFreeSpace(bytes.limit)
- val entry = new Entry(bytes, bytes.limit, false)
- memoryStore.synchronized { memoryStore.put(blockId, entry) }
- currentMemory += bytes.limit
- logDebug("Block " + blockId + " stored as " + bytes.limit + " bytes to memory")
- return Right(bytes)
- }
- }
-
- def getBytes(blockId: String): Option[ByteBuffer] = {
- throw new UnsupportedOperationException("Not implemented")
- }
-
- def getValues(blockId: String): Option[Iterator[Any]] = {
- val entry = memoryStore.synchronized { memoryStore.get(blockId) }
- if (entry == null) {
- return None
- }
- if (entry.deserialized) {
- return Some(entry.value.asInstanceOf[ArrayBuffer[Any]].toIterator)
- } else {
- return Some(dataDeserialize(entry.value.asInstanceOf[ByteBuffer].duplicate()))
- }
- }
-
- def remove(blockId: String) {
- memoryStore.synchronized {
- val entry = memoryStore.get(blockId)
- if (entry != null) {
- memoryStore.remove(blockId)
- currentMemory -= entry.size
- logDebug("Block " + blockId + " of size " + entry.size + " dropped from memory")
- } else {
- logWarning("Block " + blockId + " could not be removed as it doesnt exist")
- }
- }
- }
-
- override def clear() {
- memoryStore.synchronized {
- memoryStore.clear()
- }
- //blockDropper.shutdown()
- blockDropper.interrupt()
- logInfo("MemoryStore cleared")
- }
-
- private def ensureFreeSpace(space: Long) {
- logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format(
- space, currentMemory, maxMemory))
-
- if (maxMemory - currentMemory < space) {
-
- val selectedBlocks = new ArrayBuffer[String]()
- var selectedMemory = 0L
-
- memoryStore.synchronized {
- val iter = memoryStore.entrySet().iterator()
- while (maxMemory - (currentMemory - selectedMemory) < space && iter.hasNext) {
- val pair = iter.next()
- val blockId = pair.getKey
- val entry = pair.getValue()
- if (!entry.dropPending) {
- selectedBlocks += blockId
- entry.dropPending = true
- }
- selectedMemory += pair.getValue.size
- logDebug("Block " + blockId + " selected for dropping")
- }
- }
-
- logDebug("" + selectedBlocks.size + " new blocks selected for dropping, " +
- blocksToDrop.size + " blocks pending")
- var i = 0
- while (i < selectedBlocks.size) {
- blocksToDrop.add(selectedBlocks(i))
- i += 1
- }
- selectedBlocks.clear()
- }
- }
-}
-
-
-/**
- * Class to store blocks in disk
- */
-class DiskStore(blockManager: BlockManager, rootDirs: String)
- extends BlockStore(blockManager) {
-
- val MAX_DIR_CREATION_ATTEMPTS: Int = 10
- val localDirs = createLocalDirs()
- var lastLocalDirUsed = 0
-
- addShutdownHook()
-
- def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
- logDebug("Attempting to put block " + blockId)
- val startTime = System.currentTimeMillis
- val file = createFile(blockId)
- if (file != null) {
- val channel = new RandomAccessFile(file, "rw").getChannel()
- val buffer = channel.map(MapMode.READ_WRITE, 0, bytes.limit)
- buffer.put(bytes)
- channel.close()
- val finishTime = System.currentTimeMillis
- logDebug("Block " + blockId + " stored to file of " + bytes.limit + " bytes to disk in " + (finishTime - startTime) + " ms")
- } else {
- logError("File not created for block " + blockId)
- }
- }
-
- def putValues(blockId: String, values: Iterator[Any], level: StorageLevel): Either[Iterator[Any], ByteBuffer] = {
- val bytes = dataSerialize(values)
- logDebug("Converted block " + blockId + " to " + bytes.limit + " bytes")
- putBytes(blockId, bytes, level)
- return Right(bytes)
- }
-
- def getBytes(blockId: String): Option[ByteBuffer] = {
- val file = getFile(blockId)
- val length = file.length().toInt
- val channel = new RandomAccessFile(file, "r").getChannel()
- Some(channel.map(MapMode.READ_WRITE, 0, length))
- }
-
- def getValues(blockId: String): Option[Iterator[Any]] = {
- val file = getFile(blockId)
- val length = file.length().toInt
- val channel = new RandomAccessFile(file, "r").getChannel()
- val bytes = channel.map(MapMode.READ_ONLY, 0, length)
- val buffer = dataDeserialize(bytes)
- channel.close()
- return Some(buffer)
- }
-
- def remove(blockId: String) {
- throw new UnsupportedOperationException("Not implemented")
- }
-
- private def createFile(blockId: String): File = {
- val file = getFile(blockId)
- if (file == null) {
- lastLocalDirUsed = (lastLocalDirUsed + 1) % localDirs.size
- val newFile = new File(localDirs(lastLocalDirUsed), blockId)
- newFile.getParentFile.mkdirs()
- return newFile
- } else {
- logError("File for block " + blockId + " already exists on disk, " + file)
- return null
- }
- }
-
- private def getFile(blockId: String): File = {
- logDebug("Getting file for block " + blockId)
- // Search for the file in all the local directories, only one of them should have the file
- val files = localDirs.map(localDir => new File(localDir, blockId)).filter(_.exists)
- if (files.size > 1) {
- throw new Exception("Multiple files for same block " + blockId + " exists: " +
- files.map(_.toString).reduceLeft(_ + ", " + _))
- return null
- } else if (files.size == 0) {
- return null
- } else {
- logDebug("Got file " + files(0) + " of size " + files(0).length + " bytes")
- return files(0)
- }
- }
-
- private def createLocalDirs(): Seq[File] = {
- logDebug("Creating local directories at root dirs '" + rootDirs + "'")
- rootDirs.split("[;,:]").map(rootDir => {
- var foundLocalDir: Boolean = false
- var localDir: File = null
- var localDirUuid: UUID = null
- var tries = 0
- while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) {
- tries += 1
- try {
- localDirUuid = UUID.randomUUID()
- localDir = new File(rootDir, "spark-local-" + localDirUuid)
- if (!localDir.exists) {
- localDir.mkdirs()
- foundLocalDir = true
- }
- } catch {
- case e: Exception =>
- logWarning("Attempt " + tries + " to create local dir failed", e)
- }
- }
- if (!foundLocalDir) {
- logError("Failed " + MAX_DIR_CREATION_ATTEMPTS +
- " attempts to create local dir in " + rootDir)
- System.exit(1)
- }
- logDebug("Created local directory at " + localDir)
- localDir
- })
- }
-
- private def addShutdownHook() {
- Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") {
- override def run() {
- logDebug("Shutdown hook called")
- localDirs.foreach(localDir => Utils.deleteRecursively(localDir))
- }
- })
- }
-}
diff --git a/core/src/main/scala/spark/storage/DiskStore.scala b/core/src/main/scala/spark/storage/DiskStore.scala
new file mode 100644
index 0000000000..8ba64e4b76
--- /dev/null
+++ b/core/src/main/scala/spark/storage/DiskStore.scala
@@ -0,0 +1,180 @@
+package spark.storage
+
+import java.nio.ByteBuffer
+import java.io.{File, FileOutputStream, RandomAccessFile}
+import java.nio.channels.FileChannel.MapMode
+import java.util.{Random, Date}
+import java.text.SimpleDateFormat
+
+import it.unimi.dsi.fastutil.io.FastBufferedOutputStream
+
+import scala.collection.mutable.ArrayBuffer
+
+import spark.Utils
+
+/**
+ * Stores BlockManager blocks on disk.
+ */
+private class DiskStore(blockManager: BlockManager, rootDirs: String)
+ extends BlockStore(blockManager) {
+
+ val MAX_DIR_CREATION_ATTEMPTS: Int = 10
+ val subDirsPerLocalDir = System.getProperty("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
+ // having really large inodes at the top level.
+ val localDirs = createLocalDirs()
+ val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
+
+ addShutdownHook()
+
+ override def getSize(blockId: String): Long = {
+ getFile(blockId).length()
+ }
+
+ override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
+ logDebug("Attempting to put block " + blockId)
+ val startTime = System.currentTimeMillis
+ val file = createFile(blockId)
+ val channel = new RandomAccessFile(file, "rw").getChannel()
+ while (bytes.remaining > 0) {
+ channel.write(bytes)
+ }
+ channel.close()
+ val finishTime = System.currentTimeMillis
+ logDebug("Block %s stored as %s file on disk in %d ms".format(
+ blockId, Utils.memoryBytesToString(bytes.limit), (finishTime - startTime)))
+ }
+
+ override def putValues(
+ blockId: String,
+ values: ArrayBuffer[Any],
+ level: StorageLevel,
+ returnValues: Boolean)
+ : PutResult = {
+
+ logDebug("Attempting to write values for block " + blockId)
+ val startTime = System.currentTimeMillis
+ val file = createFile(blockId)
+ val fileOut = blockManager.wrapForCompression(blockId,
+ new FastBufferedOutputStream(new FileOutputStream(file)))
+ val objOut = blockManager.serializer.newInstance().serializeStream(fileOut)
+ objOut.writeAll(values.iterator)
+ objOut.close()
+ val length = file.length()
+ logDebug("Block %s stored as %s file on disk in %d ms".format(
+ blockId, Utils.memoryBytesToString(length), (System.currentTimeMillis - startTime)))
+
+ if (returnValues) {
+ // Return a byte buffer for the contents of the file
+ val channel = new RandomAccessFile(file, "r").getChannel()
+ val buffer = channel.map(MapMode.READ_ONLY, 0, length)
+ channel.close()
+ PutResult(length, Right(buffer))
+ } else {
+ PutResult(length, null)
+ }
+ }
+
+ override def getBytes(blockId: String): Option[ByteBuffer] = {
+ val file = getFile(blockId)
+ val length = file.length().toInt
+ val channel = new RandomAccessFile(file, "r").getChannel()
+ val bytes = channel.map(MapMode.READ_ONLY, 0, length)
+ channel.close()
+ Some(bytes)
+ }
+
+ override def getValues(blockId: String): Option[Iterator[Any]] = {
+ getBytes(blockId).map(bytes => blockManager.dataDeserialize(blockId, bytes))
+ }
+
+ override def remove(blockId: String) {
+ val file = getFile(blockId)
+ if (file.exists()) {
+ file.delete()
+ }
+ }
+
+ override def contains(blockId: String): Boolean = {
+ getFile(blockId).exists()
+ }
+
+ private def createFile(blockId: String): File = {
+ val file = getFile(blockId)
+ if (file.exists()) {
+ throw new Exception("File for block " + blockId + " already exists on disk: " + file)
+ }
+ file
+ }
+
+ private def getFile(blockId: String): File = {
+ logDebug("Getting file for block " + blockId)
+
+ // Figure out which local directory it hashes to, and which subdirectory in that
+ val hash = math.abs(blockId.hashCode)
+ val dirId = hash % localDirs.length
+ val subDirId = (hash / localDirs.length) % subDirsPerLocalDir
+
+ // Create the subdirectory if it doesn't already exist
+ var subDir = subDirs(dirId)(subDirId)
+ if (subDir == null) {
+ subDir = subDirs(dirId).synchronized {
+ val old = subDirs(dirId)(subDirId)
+ if (old != null) {
+ old
+ } else {
+ val newDir = new File(localDirs(dirId), "%02x".format(subDirId))
+ newDir.mkdir()
+ subDirs(dirId)(subDirId) = newDir
+ newDir
+ }
+ }
+ }
+
+ new File(subDir, blockId)
+ }
+
+ private def createLocalDirs(): Array[File] = {
+ logDebug("Creating local directories at root dirs '" + rootDirs + "'")
+ val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss")
+ rootDirs.split(",").map(rootDir => {
+ var foundLocalDir: Boolean = false
+ var localDir: File = null
+ var localDirId: String = null
+ var tries = 0
+ val rand = new Random()
+ while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) {
+ tries += 1
+ try {
+ localDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536))
+ localDir = new File(rootDir, "spark-local-" + localDirId)
+ if (!localDir.exists) {
+ localDir.mkdirs()
+ foundLocalDir = true
+ }
+ } catch {
+ case e: Exception =>
+ logWarning("Attempt " + tries + " to create local dir failed", e)
+ }
+ }
+ if (!foundLocalDir) {
+ logError("Failed " + MAX_DIR_CREATION_ATTEMPTS +
+ " attempts to create local dir in " + rootDir)
+ System.exit(1)
+ }
+ logInfo("Created local directory at " + localDir)
+ localDir
+ })
+ }
+
+ private def addShutdownHook() {
+ Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") {
+ override def run() {
+ logDebug("Shutdown hook called")
+ localDirs.foreach(localDir => Utils.deleteRecursively(localDir))
+ }
+ })
+ }
+}
diff --git a/core/src/main/scala/spark/storage/MemoryStore.scala b/core/src/main/scala/spark/storage/MemoryStore.scala
new file mode 100644
index 0000000000..773970446a
--- /dev/null
+++ b/core/src/main/scala/spark/storage/MemoryStore.scala
@@ -0,0 +1,218 @@
+package spark.storage
+
+import java.util.LinkedHashMap
+import java.util.concurrent.ArrayBlockingQueue
+import spark.{SizeEstimator, Utils}
+import java.nio.ByteBuffer
+import collection.mutable.ArrayBuffer
+
+/**
+ * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as
+ * serialized ByteBuffers.
+ */
+private class MemoryStore(blockManager: BlockManager, maxMemory: Long)
+ extends BlockStore(blockManager) {
+
+ case class Entry(value: Any, size: Long, deserialized: Boolean, var dropPending: Boolean = false)
+
+ private val entries = new LinkedHashMap[String, Entry](32, 0.75f, true)
+ private var currentMemory = 0L
+
+ logInfo("MemoryStore started with capacity %s.".format(Utils.memoryBytesToString(maxMemory)))
+
+ def freeMemory: Long = maxMemory - currentMemory
+
+ override def getSize(blockId: String): Long = {
+ synchronized {
+ entries.get(blockId).size
+ }
+ }
+
+ override def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel) {
+ if (level.deserialized) {
+ bytes.rewind()
+ val values = blockManager.dataDeserialize(blockId, bytes)
+ val elements = new ArrayBuffer[Any]
+ elements ++= values
+ val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef])
+ tryToPut(blockId, elements, sizeEstimate, true)
+ } else {
+ val entry = new Entry(bytes, bytes.limit, false)
+ ensureFreeSpace(blockId, bytes.limit)
+ synchronized { entries.put(blockId, entry) }
+ tryToPut(blockId, bytes, bytes.limit, false)
+ }
+ }
+
+ override def putValues(
+ blockId: String,
+ values: ArrayBuffer[Any],
+ level: StorageLevel,
+ returnValues: Boolean)
+ : PutResult = {
+
+ if (level.deserialized) {
+ val sizeEstimate = SizeEstimator.estimate(values.asInstanceOf[AnyRef])
+ tryToPut(blockId, values, sizeEstimate, true)
+ PutResult(sizeEstimate, Left(values.iterator))
+ } else {
+ val bytes = blockManager.dataSerialize(blockId, values.iterator)
+ tryToPut(blockId, bytes, bytes.limit, false)
+ PutResult(bytes.limit(), Right(bytes))
+ }
+ }
+
+ override def getBytes(blockId: String): Option[ByteBuffer] = {
+ val entry = synchronized {
+ entries.get(blockId)
+ }
+ if (entry == null) {
+ None
+ } else if (entry.deserialized) {
+ Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[ArrayBuffer[Any]].iterator))
+ } else {
+ Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data
+ }
+ }
+
+ override def getValues(blockId: String): Option[Iterator[Any]] = {
+ val entry = synchronized {
+ entries.get(blockId)
+ }
+ if (entry == null) {
+ None
+ } else if (entry.deserialized) {
+ Some(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)
+ } else {
+ val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data
+ Some(blockManager.dataDeserialize(blockId, buffer))
+ }
+ }
+
+ override def remove(blockId: String) {
+ synchronized {
+ val entry = entries.get(blockId)
+ if (entry != null) {
+ entries.remove(blockId)
+ currentMemory -= entry.size
+ logInfo("Block %s of size %d dropped from memory (free %d)".format(
+ blockId, entry.size, freeMemory))
+ } else {
+ logWarning("Block " + blockId + " could not be removed as it does not exist")
+ }
+ }
+ }
+
+ override def clear() {
+ synchronized {
+ entries.clear()
+ }
+ logInfo("MemoryStore cleared")
+ }
+
+ /**
+ * Return the RDD ID that a given block ID is from, or null if it is not an RDD block.
+ */
+ private def getRddId(blockId: String): String = {
+ if (blockId.startsWith("rdd_")) {
+ blockId.split('_')(1)
+ } else {
+ null
+ }
+ }
+
+ /**
+ * Try to put in a set of values, if we can free up enough space. The value should either be
+ * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated)
+ * size must also be passed by the caller.
+ */
+ private def tryToPut(blockId: String, value: Any, size: Long, deserialized: Boolean): Boolean = {
+ synchronized {
+ if (ensureFreeSpace(blockId, size)) {
+ val entry = new Entry(value, size, deserialized)
+ entries.put(blockId, entry)
+ currentMemory += size
+ if (deserialized) {
+ logInfo("Block %s stored as values to memory (estimated size %s, free %s)".format(
+ blockId, Utils.memoryBytesToString(size), Utils.memoryBytesToString(freeMemory)))
+ } else {
+ logInfo("Block %s stored as bytes to memory (size %s, free %s)".format(
+ blockId, Utils.memoryBytesToString(size), Utils.memoryBytesToString(freeMemory)))
+ }
+ true
+ } else {
+ // Tell the block manager that we couldn't put it in memory so that it can drop it to
+ // disk if the block allows disk storage.
+ val data = if (deserialized) {
+ Left(value.asInstanceOf[ArrayBuffer[Any]])
+ } else {
+ Right(value.asInstanceOf[ByteBuffer].duplicate())
+ }
+ blockManager.dropFromMemory(blockId, data)
+ false
+ }
+ }
+ }
+
+ /**
+ * Tries to free up a given amount of space to store a particular block, but can fail and return
+ * false if either the block is bigger than our memory or it would require replacing another
+ * block from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that
+ * don't fit into memory that we want to avoid).
+ *
+ * Assumes that a lock on the MemoryStore is held by the caller. (Otherwise, the freed space
+ * might fill up before the caller puts in their new value.)
+ */
+ private def ensureFreeSpace(blockIdToAdd: String, space: Long): Boolean = {
+ logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format(
+ space, currentMemory, maxMemory))
+
+ if (space > maxMemory) {
+ logInfo("Will not store " + blockIdToAdd + " as it is larger than our memory limit")
+ return false
+ }
+
+ // TODO: This should relinquish the lock on the MemoryStore while flushing out old blocks
+ // in order to allow parallelism in writing to disk
+ if (maxMemory - currentMemory < space) {
+ val rddToAdd = getRddId(blockIdToAdd)
+ val selectedBlocks = new ArrayBuffer[String]()
+ var selectedMemory = 0L
+
+ val iterator = entries.entrySet().iterator()
+ while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) {
+ val pair = iterator.next()
+ val blockId = pair.getKey
+ if (rddToAdd != null && rddToAdd == getRddId(blockId)) {
+ logInfo("Will not store " + blockIdToAdd + " as it would require dropping another " +
+ "block from the same RDD")
+ return false
+ }
+ selectedBlocks += blockId
+ selectedMemory += pair.getValue.size
+ }
+
+ if (maxMemory - (currentMemory - selectedMemory) >= space) {
+ logInfo(selectedBlocks.size + " blocks selected for dropping")
+ for (blockId <- selectedBlocks) {
+ val entry = entries.get(blockId)
+ val data = if (entry.deserialized) {
+ Left(entry.value.asInstanceOf[ArrayBuffer[Any]])
+ } else {
+ Right(entry.value.asInstanceOf[ByteBuffer].duplicate())
+ }
+ blockManager.dropFromMemory(blockId, data)
+ }
+ return true
+ } else {
+ return false
+ }
+ }
+ return true
+ }
+
+ override def contains(blockId: String): Boolean = {
+ synchronized { entries.containsKey(blockId) }
+ }
+}
+
diff --git a/core/src/main/scala/spark/storage/PutResult.scala b/core/src/main/scala/spark/storage/PutResult.scala
new file mode 100644
index 0000000000..76f236057b
--- /dev/null
+++ b/core/src/main/scala/spark/storage/PutResult.scala
@@ -0,0 +1,9 @@
+package spark.storage
+
+import java.nio.ByteBuffer
+
+/**
+ * Result of adding a block into a BlockStore. Contains its estimated size, and possibly the
+ * values put if the caller asked for them to be returned (e.g. for chaining replication)
+ */
+private[spark] case class PutResult(size: Long, data: Either[Iterator[_], ByteBuffer])
diff --git a/core/src/main/scala/spark/storage/StorageLevel.scala b/core/src/main/scala/spark/storage/StorageLevel.scala
index a64393eba7..c497f03e0c 100644
--- a/core/src/main/scala/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/spark/storage/StorageLevel.scala
@@ -1,7 +1,14 @@
package spark.storage
-import java.io._
+import java.io.{Externalizable, ObjectInput, ObjectOutput}
+/**
+ * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory,
+ * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory
+ * in a serialized format, and whether to replicate the RDD partitions on multiple nodes.
+ * The [[spark.storage.StorageLevel$]] singleton object contains some static constants for
+ * commonly useful storage levels.
+ */
class StorageLevel(
var useDisk: Boolean,
var useMemory: Boolean,
@@ -67,12 +74,12 @@ object StorageLevel {
val NONE = new StorageLevel(false, false, false)
val DISK_ONLY = new StorageLevel(true, false, false)
val DISK_ONLY_2 = new StorageLevel(true, false, false, 2)
- val MEMORY_ONLY = new StorageLevel(false, true, false)
- val MEMORY_ONLY_2 = new StorageLevel(false, true, false, 2)
- val MEMORY_ONLY_DESER = new StorageLevel(false, true, true)
- val MEMORY_ONLY_DESER_2 = new StorageLevel(false, true, true, 2)
- val DISK_AND_MEMORY = new StorageLevel(true, true, false)
- val DISK_AND_MEMORY_2 = new StorageLevel(true, true, false, 2)
- val DISK_AND_MEMORY_DESER = new StorageLevel(true, true, true)
- val DISK_AND_MEMORY_DESER_2 = new StorageLevel(true, true, true, 2)
+ val MEMORY_ONLY = new StorageLevel(false, true, true)
+ val MEMORY_ONLY_2 = new StorageLevel(false, true, true, 2)
+ val MEMORY_ONLY_SER = new StorageLevel(false, true, false)
+ val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, 2)
+ val MEMORY_AND_DISK = new StorageLevel(true, true, true)
+ val MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2)
+ val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false)
+ val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2)
}
diff --git a/core/src/main/scala/spark/util/AkkaUtils.scala b/core/src/main/scala/spark/util/AkkaUtils.scala
index df4e23bfd6..b466b5239c 100644
--- a/core/src/main/scala/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/spark/util/AkkaUtils.scala
@@ -17,12 +17,14 @@ import java.util.concurrent.TimeoutException
/**
* Various utility classes for working with Akka.
*/
-object AkkaUtils {
+private[spark] object AkkaUtils {
/**
* Creates an ActorSystem ready for remoting, with various Spark features. Returns both the
* ActorSystem itself and its port (which is hard to get from Akka).
*/
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 akkaConf = ConfigFactory.parseString("""
akka.daemonic = on
akka.event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
@@ -31,9 +33,9 @@ object AkkaUtils {
akka.remote.netty.hostname = "%s"
akka.remote.netty.port = %d
akka.remote.netty.connection-timeout = 1s
- akka.remote.netty.execution-pool-size = 8
- akka.actor.default-dispatcher.throughput = 30
- """.format(host, port))
+ akka.remote.netty.execution-pool-size = %d
+ akka.actor.default-dispatcher.throughput = %d
+ """.format(host, port, akkaThreads, akkaBatchSize))
val actorSystem = ActorSystem("spark", akkaConf, getClass.getClassLoader)
diff --git a/core/src/main/scala/spark/util/ByteBufferInputStream.scala b/core/src/main/scala/spark/util/ByteBufferInputStream.scala
index 0ce255105a..d7e67497fe 100644
--- a/core/src/main/scala/spark/util/ByteBufferInputStream.scala
+++ b/core/src/main/scala/spark/util/ByteBufferInputStream.scala
@@ -2,10 +2,19 @@ package spark.util
import java.io.InputStream
import java.nio.ByteBuffer
+import spark.storage.BlockManager
+
+/**
+ * Reads data from a ByteBuffer, and optionally cleans it up using BlockManager.dispose()
+ * at the end of the stream (e.g. to close a memory-mapped file).
+ */
+private[spark]
+class ByteBufferInputStream(private var buffer: ByteBuffer, dispose: Boolean = false)
+ extends InputStream {
-class ByteBufferInputStream(buffer: ByteBuffer) extends InputStream {
override def read(): Int = {
- if (buffer.remaining() == 0) {
+ if (buffer == null || buffer.remaining() == 0) {
+ cleanUp()
-1
} else {
buffer.get() & 0xFF
@@ -17,7 +26,8 @@ class ByteBufferInputStream(buffer: ByteBuffer) extends InputStream {
}
override def read(dest: Array[Byte], offset: Int, length: Int): Int = {
- if (buffer.remaining() == 0) {
+ if (buffer == null || buffer.remaining() == 0) {
+ cleanUp()
-1
} else {
val amountToGet = math.min(buffer.remaining(), length)
@@ -27,8 +37,27 @@ class ByteBufferInputStream(buffer: ByteBuffer) extends InputStream {
}
override def skip(bytes: Long): Long = {
- val amountToSkip = math.min(bytes, buffer.remaining).toInt
- buffer.position(buffer.position + amountToSkip)
- return amountToSkip
+ if (buffer != null) {
+ val amountToSkip = math.min(bytes, buffer.remaining).toInt
+ buffer.position(buffer.position + amountToSkip)
+ if (buffer.remaining() == 0) {
+ cleanUp()
+ }
+ amountToSkip
+ } else {
+ 0L
+ }
+ }
+
+ /**
+ * Clean up the buffer, and potentially dispose of it using BlockManager.dispose().
+ */
+ private def cleanUp() {
+ if (buffer != null) {
+ if (dispose) {
+ BlockManager.dispose(buffer)
+ }
+ buffer = null
+ }
}
}
diff --git a/core/src/main/scala/spark/util/IntParam.scala b/core/src/main/scala/spark/util/IntParam.scala
index c3ff063569..0427646747 100644
--- a/core/src/main/scala/spark/util/IntParam.scala
+++ b/core/src/main/scala/spark/util/IntParam.scala
@@ -3,7 +3,7 @@ package spark.util
/**
* An extractor object for parsing strings into integers.
*/
-object IntParam {
+private[spark] object IntParam {
def unapply(str: String): Option[Int] = {
try {
Some(str.toInt)
diff --git a/core/src/main/scala/spark/util/MemoryParam.scala b/core/src/main/scala/spark/util/MemoryParam.scala
index 4fba914afe..3726738842 100644
--- a/core/src/main/scala/spark/util/MemoryParam.scala
+++ b/core/src/main/scala/spark/util/MemoryParam.scala
@@ -6,7 +6,7 @@ import spark.Utils
* An extractor object for parsing JVM memory strings, such as "10g", into an Int representing
* the number of megabytes. Supports the same formats as Utils.memoryStringToMb.
*/
-object MemoryParam {
+private[spark] object MemoryParam {
def unapply(str: String): Option[Int] = {
try {
Some(Utils.memoryStringToMb(str))
diff --git a/core/src/main/scala/spark/util/SerializableBuffer.scala b/core/src/main/scala/spark/util/SerializableBuffer.scala
index 0830843a77..09d588fe1c 100644
--- a/core/src/main/scala/spark/util/SerializableBuffer.scala
+++ b/core/src/main/scala/spark/util/SerializableBuffer.scala
@@ -8,6 +8,7 @@ import java.nio.channels.Channels
* A wrapper around a java.nio.ByteBuffer that is serializable through Java serialization, to make
* it easier to pass ByteBuffers in case class messages.
*/
+private[spark]
class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable {
def value = buffer
diff --git a/core/src/main/scala/spark/util/StatCounter.scala b/core/src/main/scala/spark/util/StatCounter.scala
index 11d7939204..5f80180339 100644
--- a/core/src/main/scala/spark/util/StatCounter.scala
+++ b/core/src/main/scala/spark/util/StatCounter.scala
@@ -2,8 +2,10 @@ package spark.util
/**
* A class for tracking the statistics of a set of numbers (count, mean and variance) in a
- * numerically robust way. Includes support for merging two StatCounters. Based on Welford and
- * Chan's algorithms described at http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance.
+ * numerically robust way. Includes support for merging two StatCounters. Based on
+ * [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance Welford and Chan's algorithms for running variance]].
+ *
+ * @constructor Initialize the StatCounter with the given values.
*/
class StatCounter(values: TraversableOnce[Double]) extends Serializable {
private var n: Long = 0 // Running count of our values
@@ -12,8 +14,10 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
merge(values)
+ /** Initialize the StatCounter with no values. */
def this() = this(Nil)
+ /** Add a value into this StatCounter, updating the internal statistics. */
def merge(value: Double): StatCounter = {
val delta = value - mu
n += 1
@@ -22,11 +26,13 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
this
}
+ /** Add multiple values into this StatCounter, updating the internal statistics. */
def merge(values: TraversableOnce[Double]): StatCounter = {
values.foreach(v => merge(v))
this
}
+ /** Merge another StatCounter into this one, adding up the internal statistics. */
def merge(other: StatCounter): StatCounter = {
if (other == this) {
merge(other.copy()) // Avoid overwriting fields in a weird order
@@ -45,6 +51,7 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
}
}
+ /** Clone this StatCounter */
def copy(): StatCounter = {
val other = new StatCounter
other.n = n
@@ -59,6 +66,7 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
def sum: Double = n * mu
+ /** Return the variance of the values. */
def variance: Double = {
if (n == 0)
Double.NaN
@@ -66,6 +74,10 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
m2 / n
}
+ /**
+ * Return the sample variance, which corrects for bias in estimating the variance by dividing
+ * by N-1 instead of N.
+ */
def sampleVariance: Double = {
if (n <= 1)
Double.NaN
@@ -73,8 +85,13 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
m2 / (n - 1)
}
+ /** Return the standard deviation of the values. */
def stdev: Double = math.sqrt(variance)
+ /**
+ * Return the sample standard deviation of the values, which corrects for bias in estimating the
+ * variance by dividing by N-1 instead of N.
+ */
def sampleStdev: Double = math.sqrt(sampleVariance)
override def toString: String = {
@@ -83,7 +100,9 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable {
}
object StatCounter {
+ /** Build a StatCounter from a list of values. */
def apply(values: TraversableOnce[Double]) = new StatCounter(values)
+ /** Build a StatCounter from a list of values passed as variable-length arguments. */
def apply(values: Double*) = new StatCounter(values)
}
diff --git a/core/src/main/twirl/common/layout.scala.html b/core/src/main/twirl/spark/deploy/common/layout.scala.html
index b9192060aa..b9192060aa 100644
--- a/core/src/main/twirl/common/layout.scala.html
+++ b/core/src/main/twirl/spark/deploy/common/layout.scala.html
diff --git a/core/src/main/twirl/masterui/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
index 784d692fc2..784d692fc2 100644
--- a/core/src/main/twirl/masterui/executor_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
diff --git a/core/src/main/twirl/masterui/executors_table.scala.html b/core/src/main/twirl/spark/deploy/master/executors_table.scala.html
index cafc42c80e..cafc42c80e 100644
--- a/core/src/main/twirl/masterui/executors_table.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/executors_table.scala.html
diff --git a/core/src/main/twirl/masterui/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html
index 31ca8f4132..7562076b00 100644
--- a/core/src/main/twirl/masterui/index.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/index.scala.html
@@ -1,13 +1,13 @@
@(state: spark.deploy.MasterState)
@import spark.deploy.master._
-@common.html.layout(title = "Spark Master on " + state.uri) {
+@spark.deploy.common.html.layout(title = "Spark Master on " + state.uri) {
<!-- Cluster Details -->
<div class="row">
<div class="span12">
<ul class="unstyled">
- <li><strong>URI:</strong> spark://@(state.uri)</li>
+ <li><strong>URL:</strong> spark://@(state.uri)</li>
<li><strong>Number of Workers:</strong> @state.workers.size </li>
<li><strong>Cores:</strong> @state.workers.map(_.cores).sum Total, @state.workers.map(_.coresUsed).sum Used</li>
<li><strong>Memory:</strong> @state.workers.map(_.memory).sum Total, @state.workers.map(_.memoryUsed).sum Used</li>
@@ -47,4 +47,4 @@
</div>
</div>
-} \ No newline at end of file
+}
diff --git a/core/src/main/twirl/masterui/job_details.scala.html b/core/src/main/twirl/spark/deploy/master/job_details.scala.html
index 73cefb8269..dcf41c28f2 100644
--- a/core/src/main/twirl/masterui/job_details.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/job_details.scala.html
@@ -1,6 +1,6 @@
@(job: spark.deploy.master.JobInfo)
-@common.html.layout(title = "Job Details") {
+@spark.deploy.common.html.layout(title = "Job Details") {
<!-- Job Details -->
<div class="row">
@@ -37,4 +37,4 @@
</div>
</div>
-} \ No newline at end of file
+}
diff --git a/core/src/main/twirl/masterui/job_row.scala.html b/core/src/main/twirl/spark/deploy/master/job_row.scala.html
index 7c4865bb6e..7c4865bb6e 100644
--- a/core/src/main/twirl/masterui/job_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/job_row.scala.html
diff --git a/core/src/main/twirl/masterui/job_table.scala.html b/core/src/main/twirl/spark/deploy/master/job_table.scala.html
index 52bad6c4b8..52bad6c4b8 100644
--- a/core/src/main/twirl/masterui/job_table.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/job_table.scala.html
diff --git a/core/src/main/twirl/masterui/worker_row.scala.html b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html
index b21bd9c977..017cc4859e 100644
--- a/core/src/main/twirl/masterui/worker_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/worker_row.scala.html
@@ -4,7 +4,8 @@
<td>
<a href="http://@worker.host:@worker.webUiPort">@worker.id</href>
</td>
- <td>@worker.host:@worker.port</td>
+ <td>@{worker.host}:@{worker.port}</td>
<td>@worker.cores (@worker.coresUsed Used)</td>
- <td>@worker.memory (@worker.memoryUsed Used)</td>
-</tr> \ No newline at end of file
+ <td>@{spark.Utils.memoryMegabytesToString(worker.memory)}
+ (@{spark.Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</td>
+</tr>
diff --git a/core/src/main/twirl/masterui/worker_table.scala.html b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html
index 2028842297..2028842297 100644
--- a/core/src/main/twirl/masterui/worker_table.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/worker_table.scala.html
diff --git a/core/src/main/twirl/workerui/executor_row.scala.html b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
index c3842dbf85..c3842dbf85 100644
--- a/core/src/main/twirl/workerui/executor_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
diff --git a/core/src/main/twirl/workerui/executors_table.scala.html b/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html
index 327a2399c7..327a2399c7 100644
--- a/core/src/main/twirl/workerui/executors_table.scala.html
+++ b/core/src/main/twirl/spark/deploy/worker/executors_table.scala.html
diff --git a/core/src/main/twirl/workerui/index.scala.html b/core/src/main/twirl/spark/deploy/worker/index.scala.html
index edd82e02f2..69746ed02c 100644
--- a/core/src/main/twirl/workerui/index.scala.html
+++ b/core/src/main/twirl/spark/deploy/worker/index.scala.html
@@ -1,6 +1,6 @@
@(worker: spark.deploy.WorkerState)
-@common.html.layout(title = "Spark Worker on " + worker.uri) {
+@spark.deploy.common.html.layout(title = "Spark Worker on " + worker.uri) {
<!-- Worker Details -->
<div class="row">
@@ -12,7 +12,8 @@
(WebUI at <a href="@worker.masterWebUiUrl">@worker.masterWebUiUrl</a>)
</li>
<li><strong>Cores:</strong> @worker.cores (@worker.coresUsed Used)</li>
- <li><strong>Memory:</strong> @worker.memory (@worker.memoryUsed Used)</li>
+ <li><strong>Memory:</strong> @{spark.Utils.memoryMegabytesToString(worker.memory)}
+ (@{spark.Utils.memoryMegabytesToString(worker.memoryUsed)} Used)</li>
</ul>
</div>
</div>
@@ -39,4 +40,4 @@
</div>
</div>
-} \ No newline at end of file
+}
diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties
index 02fe16866e..4c99e450bc 100644
--- a/core/src/test/resources/log4j.properties
+++ b/core/src/test/resources/log4j.properties
@@ -1,8 +1,10 @@
# Set everything to be logged to the console
-log4j.rootCategory=WARN, 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.SSS} %p %c{1}: %m%n
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=spark-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/core/src/test/resources/uncommons-maths-1.2.2.jar b/core/src/test/resources/uncommons-maths-1.2.2.jar
new file mode 100644
index 0000000000..e126001c1c
--- /dev/null
+++ b/core/src/test/resources/uncommons-maths-1.2.2.jar
Binary files differ
diff --git a/core/src/test/scala/spark/AccumulatorSuite.scala b/core/src/test/scala/spark/AccumulatorSuite.scala
index d55969c261..d8be99dde7 100644
--- a/core/src/test/scala/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/spark/AccumulatorSuite.scala
@@ -18,6 +18,8 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
test ("basic accumulation"){
@@ -53,10 +55,11 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
}
sc.stop()
sc = null
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
}
-
implicit object SetAccum extends AccumulableParam[mutable.Set[Any], Any] {
def addInPlace(t1: mutable.Set[Any], t2: mutable.Set[Any]) : mutable.Set[Any] = {
t1 ++= t2
@@ -71,7 +74,6 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
}
}
-
test ("value not readable in tasks") {
import SetAccum._
val maxI = 1000
@@ -86,6 +88,54 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
} should produce [SparkException]
sc.stop()
sc = null
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+ }
+
+ test ("collection accumulators") {
+ val maxI = 1000
+ for (nThreads <- List(1, 10)) {
+ // test single & multi-threaded
+ sc = new SparkContext("local[" + nThreads + "]", "test")
+ val setAcc = sc.accumulableCollection(mutable.HashSet[Int]())
+ val bufferAcc = sc.accumulableCollection(mutable.ArrayBuffer[Int]())
+ val mapAcc = sc.accumulableCollection(mutable.HashMap[Int,String]())
+ val d = sc.parallelize((1 to maxI) ++ (1 to maxI))
+ d.foreach {
+ x => {setAcc += x; bufferAcc += x; mapAcc += (x -> x.toString)}
+ }
+
+ // Note that this is typed correctly -- no casts necessary
+ setAcc.value.size should be (maxI)
+ bufferAcc.value.size should be (2 * maxI)
+ mapAcc.value.size should be (maxI)
+ for (i <- 1 to maxI) {
+ setAcc.value should contain(i)
+ bufferAcc.value should contain(i)
+ mapAcc.value should contain (i -> i.toString)
+ }
+ sc.stop()
+ sc = null
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+ }
+
+ 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")
+ val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]())
+ val groupedInts = (1 to (maxI/20)).map {x => (20 * (x - 1) to 20 * x).toSet}
+ val d = sc.parallelize(groupedInts)
+ d.foreach {
+ x => acc.localValue ++= x
+ }
+ acc.value should be ( (0 to maxI).toSet)
+ sc.stop()
+ sc = null
}
}
diff --git a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
index dff2970566..37cafd1e8e 100644
--- a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
+++ b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala
@@ -2,8 +2,10 @@ package spark
import org.scalatest.FunSuite
import org.scalatest.PrivateMethodTester
+import org.scalatest.matchers.ShouldMatchers
-class BoundedMemoryCacheSuite extends FunSuite with PrivateMethodTester {
+// TODO: Replace this with a test of MemoryStore
+class BoundedMemoryCacheSuite extends FunSuite with PrivateMethodTester with ShouldMatchers {
test("constructor test") {
val cache = new BoundedMemoryCache(60)
expect(60)(cache.getCapacity)
@@ -22,15 +24,21 @@ class BoundedMemoryCacheSuite extends FunSuite with PrivateMethodTester {
logInfo("Dropping key (%s, %d) of size %d to make space".format(datasetId, partition, entry.size))
}
}
+
+ // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length
+ // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6.
+ // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html
+ // Work around to check for either.
+
//should be OK
- expect(CachePutSuccess(56))(cache.put("1", 0, "Meh"))
+ cache.put("1", 0, "Meh") should (equal (CachePutSuccess(56)) or equal (CachePutSuccess(48)))
//we cannot add this to cache (there is not enough space in cache) & we cannot evict the only value from
//cache because it's from the same dataset
expect(CachePutFailure())(cache.put("1", 1, "Meh"))
//should be OK, dataset '1' can be evicted from cache
- expect(CachePutSuccess(56))(cache.put("2", 0, "Meh"))
+ cache.put("2", 0, "Meh") should (equal (CachePutSuccess(56)) or equal (CachePutSuccess(48)))
//should fail, cache should obey it's capacity
expect(CachePutFailure())(cache.put("3", 0, "Very_long_and_useless_string"))
diff --git a/core/src/test/scala/spark/BroadcastSuite.scala b/core/src/test/scala/spark/BroadcastSuite.scala
index 0738a2725b..2d3302f0aa 100644
--- a/core/src/test/scala/spark/BroadcastSuite.scala
+++ b/core/src/test/scala/spark/BroadcastSuite.scala
@@ -12,6 +12,8 @@ class BroadcastSuite extends FunSuite with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
test("basic broadcast") {
diff --git a/core/src/test/scala/spark/CacheTrackerSuite.scala b/core/src/test/scala/spark/CacheTrackerSuite.scala
index 426c0d26e9..467605981b 100644
--- a/core/src/test/scala/spark/CacheTrackerSuite.scala
+++ b/core/src/test/scala/spark/CacheTrackerSuite.scala
@@ -22,7 +22,7 @@ class CacheTrackerSuite extends FunSuite {
} catch {
case e: Exception =>
throw new SparkException("Error communicating with actor", e)
- }
+ }
}
test("CacheTrackerActor slave initialization & cache status") {
diff --git a/core/src/test/scala/spark/DistributedSuite.scala b/core/src/test/scala/spark/DistributedSuite.scala
new file mode 100644
index 0000000000..cacc2796b6
--- /dev/null
+++ b/core/src/test/scala/spark/DistributedSuite.scala
@@ -0,0 +1,191 @@
+package spark
+
+import org.scalatest.FunSuite
+import org.scalatest.BeforeAndAfter
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.prop.Checkers
+import org.scalacheck.Arbitrary._
+import org.scalacheck.Gen
+import org.scalacheck.Prop._
+
+import com.google.common.io.Files
+
+import scala.collection.mutable.ArrayBuffer
+
+import SparkContext._
+import storage.StorageLevel
+
+class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
+
+ val clusterUrl = "local-cluster[2,1,512]"
+
+ @transient var sc: SparkContext = _
+
+ after {
+ if (sc != null) {
+ sc.stop()
+ sc = null
+ }
+ System.clearProperty("spark.reducer.maxMbInFlight")
+ System.clearProperty("spark.storage.memoryFraction")
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+
+ test("local-cluster format") {
+ sc = new SparkContext("local-cluster[2,1,512]", "test")
+ assert(sc.parallelize(1 to 2, 2).count() == 2)
+ sc.stop()
+ System.clearProperty("spark.master.port")
+ sc = new SparkContext("local-cluster[2 , 1 , 512]", "test")
+ assert(sc.parallelize(1 to 2, 2).count() == 2)
+ sc.stop()
+ System.clearProperty("spark.master.port")
+ sc = new SparkContext("local-cluster[2, 1, 512]", "test")
+ assert(sc.parallelize(1 to 2, 2).count() == 2)
+ sc.stop()
+ System.clearProperty("spark.master.port")
+ sc = new SparkContext("local-cluster[ 2, 1, 512 ]", "test")
+ assert(sc.parallelize(1 to 2, 2).count() == 2)
+ sc.stop()
+ System.clearProperty("spark.master.port")
+ sc = null
+ }
+
+ test("simple groupByKey") {
+ sc = new SparkContext(clusterUrl, "test")
+ val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 5)
+ val groups = pairs.groupByKey(5).collect()
+ assert(groups.size === 2)
+ val valuesFor1 = groups.find(_._1 == 1).get._2
+ assert(valuesFor1.toList.sorted === List(1, 2, 3))
+ val valuesFor2 = groups.find(_._1 == 2).get._2
+ assert(valuesFor2.toList.sorted === List(1))
+ }
+
+ test("groupByKey where map output sizes exceed maxMbInFlight") {
+ System.setProperty("spark.reducer.maxMbInFlight", "1")
+ sc = new SparkContext(clusterUrl, "test")
+ // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output
+ // file should be about 2.5 MB
+ val pairs = sc.parallelize(1 to 2000, 4).map(x => (x % 16, new Array[Byte](10000)))
+ val groups = pairs.groupByKey(2).map(x => (x._1, x._2.size)).collect()
+ assert(groups.length === 16)
+ assert(groups.map(_._2).sum === 2000)
+ // Note that spark.reducer.maxMbInFlight will be cleared in the test suite's after{} block
+ }
+
+ test("accumulators") {
+ sc = new SparkContext(clusterUrl, "test")
+ val accum = sc.accumulator(0)
+ sc.parallelize(1 to 10, 10).foreach(x => accum += x)
+ assert(accum.value === 55)
+ }
+
+ test("broadcast variables") {
+ sc = new SparkContext(clusterUrl, "test")
+ val array = new Array[Int](100)
+ val bv = sc.broadcast(array)
+ array(2) = 3 // Change the array -- this should not be seen on workers
+ val rdd = sc.parallelize(1 to 10, 10)
+ val sum = rdd.map(x => bv.value.sum).reduce(_ + _)
+ assert(sum === 0)
+ }
+
+ test("repeatedly failing task") {
+ sc = new SparkContext(clusterUrl, "test")
+ val accum = sc.accumulator(0)
+ val thrown = intercept[SparkException] {
+ sc.parallelize(1 to 10, 10).foreach(x => println(x / 0))
+ }
+ assert(thrown.getClass === classOf[SparkException])
+ assert(thrown.getMessage.contains("more than 4 times"))
+ }
+
+ test("caching") {
+ sc = new SparkContext(clusterUrl, "test")
+ val data = sc.parallelize(1 to 1000, 10).cache()
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ }
+
+ test("caching on disk") {
+ sc = new SparkContext(clusterUrl, "test")
+ val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ }
+
+ test("caching in memory, replicated") {
+ sc = new SparkContext(clusterUrl, "test")
+ val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_2)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ }
+
+ test("caching in memory, serialized, replicated") {
+ sc = new SparkContext(clusterUrl, "test")
+ val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_ONLY_SER_2)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ }
+
+ test("caching on disk, replicated") {
+ sc = new SparkContext(clusterUrl, "test")
+ val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.DISK_ONLY_2)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ }
+
+ test("caching in memory and disk, replicated") {
+ sc = new SparkContext(clusterUrl, "test")
+ val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_2)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ }
+
+ test("caching in memory and disk, serialized, replicated") {
+ sc = new SparkContext(clusterUrl, "test")
+ val data = sc.parallelize(1 to 1000, 10).persist(StorageLevel.MEMORY_AND_DISK_SER_2)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ assert(data.count() === 1000)
+ }
+
+ test("compute without caching when no partitions fit in memory") {
+ System.setProperty("spark.storage.memoryFraction", "0.0001")
+ sc = new SparkContext(clusterUrl, "test")
+ // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache
+ // to only 50 KB (0.0001 of 512 MB), so no partitions should fit in memory
+ val data = sc.parallelize(1 to 4000000, 2).persist(StorageLevel.MEMORY_ONLY_SER)
+ assert(data.count() === 4000000)
+ assert(data.count() === 4000000)
+ assert(data.count() === 4000000)
+ System.clearProperty("spark.storage.memoryFraction")
+ }
+
+ test("compute when only some partitions fit in memory") {
+ System.setProperty("spark.storage.memoryFraction", "0.01")
+ sc = new SparkContext(clusterUrl, "test")
+ // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache
+ // to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions
+ // to make sure that *some* of them do fit though
+ val data = sc.parallelize(1 to 4000000, 20).persist(StorageLevel.MEMORY_ONLY_SER)
+ assert(data.count() === 4000000)
+ assert(data.count() === 4000000)
+ assert(data.count() === 4000000)
+ System.clearProperty("spark.storage.memoryFraction")
+ }
+
+ test("passing environment variables to cluster") {
+ sc = new SparkContext(clusterUrl, "test", null, Nil, Map("TEST_VAR" -> "TEST_VALUE"))
+ val values = sc.parallelize(1 to 2, 2).map(x => System.getenv("TEST_VAR")).collect()
+ assert(values.toSeq === Seq("TEST_VALUE", "TEST_VALUE"))
+ }
+}
diff --git a/core/src/test/scala/spark/FailureSuite.scala b/core/src/test/scala/spark/FailureSuite.scala
index 4405829161..a3454f25f6 100644
--- a/core/src/test/scala/spark/FailureSuite.scala
+++ b/core/src/test/scala/spark/FailureSuite.scala
@@ -32,6 +32,8 @@ class FailureSuite extends FunSuite with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
// Run a 3-task map job in which task 1 deterministically fails once, and check
diff --git a/core/src/test/scala/spark/FileServerSuite.scala b/core/src/test/scala/spark/FileServerSuite.scala
new file mode 100644
index 0000000000..b4283d9604
--- /dev/null
+++ b/core/src/test/scala/spark/FileServerSuite.scala
@@ -0,0 +1,110 @@
+package spark
+
+import com.google.common.io.Files
+import org.scalatest.FunSuite
+import org.scalatest.BeforeAndAfter
+import java.io.{File, PrintWriter, FileReader, BufferedReader}
+import SparkContext._
+
+class FileServerSuite extends FunSuite with BeforeAndAfter {
+
+ @transient var sc: SparkContext = _
+ @transient var tmpFile : File = _
+ @transient var testJarFile : File = _
+
+ before {
+ // 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)
+ pw.println("100")
+ pw.close()
+ }
+
+ after {
+ if (sc != null) {
+ sc.stop()
+ sc = null
+ }
+ // Clean up downloaded file
+ if (tmpFile.exists) {
+ tmpFile.delete()
+ }
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
+ }
+
+ test("Distributing files locally") {
+ sc = new SparkContext("local[4]", "test")
+ sc.addFile(tmpFile.toString)
+ val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
+ val result = sc.parallelize(testData).reduceByKey {
+ val in = new BufferedReader(new FileReader("FileServerSuite.txt"))
+ val fileVal = in.readLine().toInt
+ in.close()
+ _ * fileVal + _ * fileVal
+ }.collect
+ assert(result.toSet === Set((1,200), (2,300), (3,500)))
+ }
+
+ test("Distributing files locally using URL as input") {
+ // addFile("file:///....")
+ sc = new SparkContext("local[4]", "test")
+ sc.addFile((new File(tmpFile.toString)).toURL.toString)
+ val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
+ val result = sc.parallelize(testData).reduceByKey {
+ val in = new BufferedReader(new FileReader("FileServerSuite.txt"))
+ val fileVal = in.readLine().toInt
+ in.close()
+ _ * fileVal + _ * fileVal
+ }.collect
+ assert(result.toSet === Set((1,200), (2,300), (3,500)))
+ }
+
+ 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)))
+ }
+
+ test("Distributing files on a standalone cluster") {
+ sc = new SparkContext("local-cluster[1,1,512]", "test")
+ sc.addFile(tmpFile.toString)
+ val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0))
+ val result = sc.parallelize(testData).reduceByKey {
+ val in = new BufferedReader(new FileReader("FileServerSuite.txt"))
+ val fileVal = in.readLine().toInt
+ in.close()
+ _ * fileVal + _ * fileVal
+ }.collect
+ println(result)
+ assert(result.toSet === Set((1,200), (2,300), (3,500)))
+ }
+
+ 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)))
+ }
+}
diff --git a/core/src/test/scala/spark/FileSuite.scala b/core/src/test/scala/spark/FileSuite.scala
index 17c7a8de43..554bea53a9 100644
--- a/core/src/test/scala/spark/FileSuite.scala
+++ b/core/src/test/scala/spark/FileSuite.scala
@@ -1,6 +1,6 @@
package spark
-import java.io.File
+import java.io.{FileWriter, PrintWriter, File}
import scala.io.Source
@@ -20,6 +20,8 @@ class FileSuite extends FunSuite with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
test("text files") {
@@ -142,4 +144,18 @@ class FileSuite extends FunSuite with BeforeAndAfter {
sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir)
assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)"))
}
+
+ test("file caching") {
+ sc = new SparkContext("local", "test")
+ 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 rdd = sc.textFile(tempDir + "/input").cache()
+ assert(rdd.count() === 3)
+ assert(rdd.count() === 3)
+ assert(rdd.count() === 3)
+ }
}
diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java
index 24bf021710..5875506179 100644
--- a/core/src/test/scala/spark/JavaAPISuite.java
+++ b/core/src/test/scala/spark/JavaAPISuite.java
@@ -112,15 +112,19 @@ public class JavaAPISuite implements Serializable {
Assert.assertEquals(new Tuple2<Integer, Integer>(3, 2), sortedPairs.get(2));
}
+ static int foreachCalls = 0;
+
@Test
public void foreach() {
+ foreachCalls = 0;
JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World"));
rdd.foreach(new VoidFunction<String>() {
@Override
public void call(String s) {
- System.out.println(s);
+ foreachCalls++;
}
});
+ Assert.assertEquals(2, foreachCalls);
}
@Test
diff --git a/core/src/test/scala/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/spark/MapOutputTrackerSuite.scala
new file mode 100644
index 0000000000..4e9717d871
--- /dev/null
+++ b/core/src/test/scala/spark/MapOutputTrackerSuite.scala
@@ -0,0 +1,25 @@
+package spark
+
+import org.scalatest.FunSuite
+
+class MapOutputTrackerSuite extends FunSuite {
+ test("compressSize") {
+ assert(MapOutputTracker.compressSize(0L) === 0)
+ assert(MapOutputTracker.compressSize(1L) === 0)
+ assert(MapOutputTracker.compressSize(2L) === 8)
+ assert(MapOutputTracker.compressSize(10L) === 25)
+ assert((MapOutputTracker.compressSize(1000000L) & 0xFF) === 145)
+ assert((MapOutputTracker.compressSize(1000000000L) & 0xFF) === 218)
+ // This last size is bigger than we can encode in a byte, so check that we just return 255
+ assert((MapOutputTracker.compressSize(1000000000000000000L) & 0xFF) === 255)
+ }
+
+ test("decompressSize") {
+ assert(MapOutputTracker.decompressSize(0) === 1)
+ for (size <- Seq(2L, 10L, 100L, 50000L, 1000000L, 1000000000L)) {
+ val size2 = MapOutputTracker.decompressSize(MapOutputTracker.compressSize(size))
+ assert(size2 >= 0.99 * size && size2 <= 1.11 * size,
+ "size " + size + " decompressed to " + size2 + ", which is out of range")
+ }
+ }
+}
diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala
index 5000fa7307..3dadc7acec 100644
--- a/core/src/test/scala/spark/PartitioningSuite.scala
+++ b/core/src/test/scala/spark/PartitioningSuite.scala
@@ -16,6 +16,8 @@ class PartitioningSuite extends FunSuite with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
diff --git a/core/src/test/scala/spark/PipedRDDSuite.scala b/core/src/test/scala/spark/PipedRDDSuite.scala
index 426652dc15..9b84b29227 100644
--- a/core/src/test/scala/spark/PipedRDDSuite.scala
+++ b/core/src/test/scala/spark/PipedRDDSuite.scala
@@ -13,6 +13,8 @@ class PipedRDDSuite extends FunSuite with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
test("basic pipe") {
@@ -22,7 +24,6 @@ class PipedRDDSuite extends FunSuite with BeforeAndAfter {
val piped = nums.pipe(Seq("cat"))
val c = piped.collect()
- println(c.toSeq)
assert(c.size === 4)
assert(c(0) === "1")
assert(c(1) === "2")
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index ba9b36adb7..37a0ff0947 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -3,6 +3,8 @@ package spark
import scala.collection.mutable.HashMap
import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfter
+
+import spark.rdd.CoalescedRDD
import SparkContext._
class RDDSuite extends FunSuite with BeforeAndAfter {
@@ -14,6 +16,8 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
test("basic operations") {
@@ -29,6 +33,11 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4)))
val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _)))
assert(partitionSums.collect().toList === List(3, 7))
+
+ val partitionSumsWithSplit = nums.mapPartitionsWithSplit {
+ case(split, iter) => Iterator((split, iter.reduceLeft(_ + _)))
+ }
+ assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7)))
}
test("SparkContext.union") {
@@ -66,4 +75,43 @@ class RDDSuite extends FunSuite with BeforeAndAfter {
val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).flatMap(x => 1 to x).checkpoint()
assert(rdd.collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4))
}
+
+ test("basic caching") {
+ sc = new SparkContext("local", "test")
+ val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache()
+ assert(rdd.collect().toList === List(1, 2, 3, 4))
+ assert(rdd.collect().toList === List(1, 2, 3, 4))
+ assert(rdd.collect().toList === List(1, 2, 3, 4))
+ }
+
+ test("coalesced RDDs") {
+ sc = new SparkContext("local", "test")
+ val data = sc.parallelize(1 to 10, 10)
+
+ val coalesced1 = new CoalescedRDD(data, 2)
+ assert(coalesced1.collect().toList === (1 to 10).toList)
+ assert(coalesced1.glom().collect().map(_.toList).toList ===
+ List(List(1, 2, 3, 4, 5), List(6, 7, 8, 9, 10)))
+
+ // Check that the narrow dependency is also specified correctly
+ assert(coalesced1.dependencies.head.getParents(0).toList === List(0, 1, 2, 3, 4))
+ assert(coalesced1.dependencies.head.getParents(1).toList === List(5, 6, 7, 8, 9))
+
+ val coalesced2 = new CoalescedRDD(data, 3)
+ assert(coalesced2.collect().toList === (1 to 10).toList)
+ assert(coalesced2.glom().collect().map(_.toList).toList ===
+ List(List(1, 2, 3), List(4, 5, 6), List(7, 8, 9, 10)))
+
+ val coalesced3 = new CoalescedRDD(data, 10)
+ assert(coalesced3.collect().toList === (1 to 10).toList)
+ assert(coalesced3.glom().collect().map(_.toList).toList ===
+ (1 to 10).map(x => List(x)).toList)
+
+ // If we try to coalesce into more partitions than the original RDD, it should just
+ // keep the original number of partitions.
+ val coalesced4 = new CoalescedRDD(data, 20)
+ assert(coalesced4.collect().toList === (1 to 10).toList)
+ assert(coalesced4.glom().collect().map(_.toList).toList ===
+ (1 to 10).map(x => List(x)).toList)
+ }
}
diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index 99d13b31ef..8170100f1d 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -1,7 +1,10 @@
package spark
+import scala.collection.mutable.ArrayBuffer
+
import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfter
+import org.scalatest.matchers.ShouldMatchers
import org.scalatest.prop.Checkers
import org.scalacheck.Arbitrary._
import org.scalacheck.Gen
@@ -9,21 +12,22 @@ import org.scalacheck.Prop._
import com.google.common.io.Files
-import scala.collection.mutable.ArrayBuffer
+import spark.rdd.ShuffledRDD
+import spark.SparkContext._
-import SparkContext._
+class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter {
-class ShuffleSuite extends FunSuite with BeforeAndAfter {
-
var sc: SparkContext = _
-
+
after {
if (sc != null) {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
-
+
test("groupByKey") {
sc = new SparkContext("local", "test")
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)))
@@ -56,7 +60,7 @@ class ShuffleSuite extends FunSuite with BeforeAndAfter {
val valuesFor2 = groups.find(_._1 == 2).get._2
assert(valuesFor2.toList.sorted === List(1))
}
-
+
test("groupByKey with many output partitions") {
sc = new SparkContext("local", "test")
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)))
@@ -68,6 +72,22 @@ class ShuffleSuite extends FunSuite with BeforeAndAfter {
assert(valuesFor2.toList.sorted === List(1))
}
+ test("groupByKey with compression") {
+ try {
+ System.setProperty("spark.blockManager.compress", "true")
+ sc = new SparkContext("local", "test")
+ val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4)
+ val groups = pairs.groupByKey(4).collect()
+ assert(groups.size === 2)
+ val valuesFor1 = groups.find(_._1 == 1).get._2
+ assert(valuesFor1.toList.sorted === List(1, 2, 3))
+ val valuesFor2 = groups.find(_._1 == 2).get._2
+ assert(valuesFor2.toList.sorted === List(1))
+ } finally {
+ System.setProperty("spark.blockManager.compress", "false")
+ }
+ }
+
test("reduceByKey") {
sc = new SparkContext("local", "test")
val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1)))
@@ -186,7 +206,7 @@ class ShuffleSuite extends FunSuite with BeforeAndAfter {
(4, (ArrayBuffer(), ArrayBuffer('w')))
))
}
-
+
test("zero-partition RDD") {
sc = new SparkContext("local", "test")
val emptyDir = Files.createTempDir()
@@ -194,6 +214,13 @@ class ShuffleSuite extends FunSuite with BeforeAndAfter {
assert(file.splits.size == 0)
assert(file.collect().toList === Nil)
// Test that a shuffle on the file works, because this used to be a bug
- assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil)
+ assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil)
+ }
+}
+
+object ShuffleSuite {
+ def mergeCombineException(x: Int, y: Int): Int = {
+ throw new SparkException("Exception for map-side combine.")
+ x + y
}
}
diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala
index a2015644ee..17f366212b 100644
--- a/core/src/test/scala/spark/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala
@@ -3,6 +3,7 @@ package spark
import org.scalatest.FunSuite
import org.scalatest.BeforeAndAfterAll
import org.scalatest.PrivateMethodTester
+import org.scalatest.matchers.ShouldMatchers
class DummyClass1 {}
@@ -19,7 +20,9 @@ class DummyClass4(val d: DummyClass3) {
val x: Int = 0
}
-class SizeEstimatorSuite extends FunSuite with BeforeAndAfterAll with PrivateMethodTester {
+class SizeEstimatorSuite
+ extends FunSuite with BeforeAndAfterAll with PrivateMethodTester with ShouldMatchers {
+
var oldArch: String = _
var oldOops: String = _
@@ -42,11 +45,15 @@ class SizeEstimatorSuite extends FunSuite with BeforeAndAfterAll with PrivateMet
expect(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3)))
}
+ // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length.
+ // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6.
+ // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html
+ // Work around to check for either.
test("strings") {
- expect(48)(SizeEstimator.estimate(""))
- expect(56)(SizeEstimator.estimate("a"))
- expect(56)(SizeEstimator.estimate("ab"))
- expect(64)(SizeEstimator.estimate("abcdefgh"))
+ SizeEstimator.estimate("") should (equal (48) or equal (40))
+ SizeEstimator.estimate("a") should (equal (56) or equal (48))
+ SizeEstimator.estimate("ab") should (equal (56) or equal (48))
+ SizeEstimator.estimate("abcdefgh") should (equal(64) or equal(56))
}
test("primitive arrays") {
@@ -106,6 +113,10 @@ class SizeEstimatorSuite extends FunSuite with BeforeAndAfterAll with PrivateMet
resetOrClear("os.arch", arch)
}
+ // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length.
+ // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6.
+ // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html
+ // Work around to check for either.
test("64-bit arch with no compressed oops") {
val arch = System.setProperty("os.arch", "amd64")
val oops = System.setProperty("spark.test.useCompressedOops", "false")
@@ -113,10 +124,10 @@ class SizeEstimatorSuite extends FunSuite with BeforeAndAfterAll with PrivateMet
val initialize = PrivateMethod[Unit]('initialize)
SizeEstimator invokePrivate initialize()
- expect(64)(SizeEstimator.estimate(""))
- expect(72)(SizeEstimator.estimate("a"))
- expect(72)(SizeEstimator.estimate("ab"))
- expect(80)(SizeEstimator.estimate("abcdefgh"))
+ SizeEstimator.estimate("") should (equal (64) or equal (56))
+ SizeEstimator.estimate("a") should (equal (72) or equal (64))
+ SizeEstimator.estimate("ab") should (equal (72) or equal (64))
+ SizeEstimator.estimate("abcdefgh") should (equal (80) or equal (72))
resetOrClear("os.arch", arch)
resetOrClear("spark.test.useCompressedOops", oops)
diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala
index 8fa1442a4d..1ad11ff4c3 100644
--- a/core/src/test/scala/spark/SortingSuite.scala
+++ b/core/src/test/scala/spark/SortingSuite.scala
@@ -12,12 +12,15 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
after {
if (sc != null) {
sc.stop()
+ sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
test("sortByKey") {
sc = new SparkContext("local", "test")
- val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)))
+ val pairs = sc.parallelize(Array((1, 0), (2, 0), (0, 0), (3, 0)), 2)
assert(pairs.sortByKey().collect() === Array((0,0), (1,0), (2,0), (3,0)))
}
@@ -25,18 +28,56 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
sc = new SparkContext("local", "test")
val rand = new scala.util.Random()
val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
- val pairs = sc.parallelize(pairArr)
- assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1))
+ val pairs = sc.parallelize(pairArr, 2)
+ val sorted = pairs.sortByKey()
+ assert(sorted.splits.size === 2)
+ assert(sorted.collect() === pairArr.sortBy(_._1))
}
+ test("large array with one split") {
+ sc = new SparkContext("local", "test")
+ val rand = new scala.util.Random()
+ val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
+ val pairs = sc.parallelize(pairArr, 2)
+ val sorted = pairs.sortByKey(true, 1)
+ assert(sorted.splits.size === 1)
+ assert(sorted.collect() === pairArr.sortBy(_._1))
+ }
+
+ test("large array with many splits") {
+ sc = new SparkContext("local", "test")
+ val rand = new scala.util.Random()
+ val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
+ val pairs = sc.parallelize(pairArr, 2)
+ val sorted = pairs.sortByKey(true, 20)
+ assert(sorted.splits.size === 20)
+ assert(sorted.collect() === pairArr.sortBy(_._1))
+ }
+
test("sort descending") {
sc = new SparkContext("local", "test")
val rand = new scala.util.Random()
val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
- val pairs = sc.parallelize(pairArr)
+ val pairs = sc.parallelize(pairArr, 2)
assert(pairs.sortByKey(false).collect() === pairArr.sortWith((x, y) => x._1 > y._1))
}
+ test("sort descending with one split") {
+ sc = new SparkContext("local", "test")
+ val rand = new scala.util.Random()
+ val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
+ val pairs = sc.parallelize(pairArr, 1)
+ assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1))
+ }
+
+ test("sort descending with many splits") {
+ sc = new SparkContext("local", "test")
+ val rand = new scala.util.Random()
+ val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
+ val pairs = sc.parallelize(pairArr, 2)
+ assert(pairs.sortByKey(false, 20).collect() === pairArr.sortWith((x, y) => x._1 > y._1))
+ }
+
test("more partitions than elements") {
sc = new SparkContext("local", "test")
val rand = new scala.util.Random()
@@ -48,7 +89,7 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
test("empty RDD") {
sc = new SparkContext("local", "test")
val pairArr = new Array[(Int, Int)](0)
- val pairs = sc.parallelize(pairArr)
+ val pairs = sc.parallelize(pairArr, 2)
assert(pairs.sortByKey().collect() === pairArr.sortBy(_._1))
}
@@ -58,11 +99,11 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
val sorted = sc.parallelize(pairArr, 4).sortByKey()
assert(sorted.collect() === pairArr.sortBy(_._1))
val partitions = sorted.collectPartitions()
- logInfo("partition lengths: " + partitions.map(_.length).mkString(", "))
- partitions(0).length should be > 200
- partitions(1).length should be > 200
- partitions(2).length should be > 200
- partitions(3).length should be > 200
+ logInfo("Partition lengths: " + partitions.map(_.length).mkString(", "))
+ partitions(0).length should be > 180
+ partitions(1).length should be > 180
+ partitions(2).length should be > 180
+ partitions(3).length should be > 180
partitions(0).last should be < partitions(1).head
partitions(1).last should be < partitions(2).head
partitions(2).last should be < partitions(3).head
@@ -75,10 +116,10 @@ class SortingSuite extends FunSuite with BeforeAndAfter with ShouldMatchers with
assert(sorted.collect() === pairArr.sortBy(_._1).reverse)
val partitions = sorted.collectPartitions()
logInfo("partition lengths: " + partitions.map(_.length).mkString(", "))
- partitions(0).length should be > 200
- partitions(1).length should be > 200
- partitions(2).length should be > 200
- partitions(3).length should be > 200
+ partitions(0).length should be > 180
+ partitions(1).length should be > 180
+ partitions(2).length should be > 180
+ partitions(3).length should be > 180
partitions(0).last should be > partitions(1).head
partitions(1).last should be > partitions(2).head
partitions(2).last should be > partitions(3).head
diff --git a/core/src/test/scala/spark/ThreadingSuite.scala b/core/src/test/scala/spark/ThreadingSuite.scala
index 302f731187..e9b1837d89 100644
--- a/core/src/test/scala/spark/ThreadingSuite.scala
+++ b/core/src/test/scala/spark/ThreadingSuite.scala
@@ -31,6 +31,8 @@ class ThreadingSuite extends FunSuite with BeforeAndAfter {
sc.stop()
sc = null
}
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
}
diff --git a/core/src/test/scala/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
index f3f891e471..b9c19e61cd 100644
--- a/core/src/test/scala/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/spark/storage/BlockManagerSuite.scala
@@ -13,10 +13,14 @@ import spark.SizeEstimator
import spark.util.ByteBufferInputStream
class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester {
+ var store: BlockManager = null
var actorSystem: ActorSystem = null
var master: BlockManagerMaster = null
- var oldArch: String = _
- var oldOops: String = _
+ var oldArch: String = null
+ var oldOops: String = null
+
+ // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
+ val serializer = new KryoSerializer
before {
actorSystem = ActorSystem("test")
@@ -30,6 +34,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
after {
+ if (store != null) {
+ store.stop()
+ }
actorSystem.shutdown()
actorSystem.awaitTermination()
actorSystem = null
@@ -49,15 +56,15 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("manager-master interaction") {
- val store = new BlockManager(master, new KryoSerializer, 2000)
+ store = new BlockManager(master, serializer, 2000)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
// Putting a1, a2 and a3 in memory and telling master only about a1 and a2
- store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER)
- store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_DESER)
- store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_DESER, false)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY)
+ store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY, false)
// Checking whether blocks are in memory
assert(store.getSingle("a1") != None, "a1 was not in store")
@@ -69,9 +76,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
assert(master.mustGetLocations(GetLocations("a2")).size > 0, "master was not told about a2")
assert(master.mustGetLocations(GetLocations("a3")).size === 0, "master was told about a3")
- // Setting storage level of a1 and a2 to invalid; they should be removed from store and master
- store.setLevel("a1", new StorageLevel(false, false, false, 1))
- store.setLevel("a2", new StorageLevel(true, false, false, 0))
+ // Drop a1 and a2 from memory; this should be reported back to the master
+ store.dropFromMemory("a1", null)
+ store.dropFromMemory("a2", null)
assert(store.getSingle("a1") === None, "a1 not removed from store")
assert(store.getSingle("a2") === None, "a2 not removed from store")
assert(master.mustGetLocations(GetLocations("a1")).size === 0, "master did not remove a1")
@@ -79,97 +86,167 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("in-memory LRU storage") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
- store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER)
- store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_DESER)
- store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_DESER)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY)
+ store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY)
assert(store.getSingle("a2") != None, "a2 was not in store")
assert(store.getSingle("a3") != None, "a3 was not in store")
- Thread.sleep(100)
assert(store.getSingle("a1") === None, "a1 was in store")
assert(store.getSingle("a2") != None, "a2 was not in store")
// At this point a2 was gotten last, so LRU will getSingle rid of a3
- store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
assert(store.getSingle("a1") != None, "a1 was not in store")
assert(store.getSingle("a2") != None, "a2 was not in store")
- Thread.sleep(100)
assert(store.getSingle("a3") === None, "a3 was in store")
}
test("in-memory LRU storage with serialization") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
- store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
- store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY)
- store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY)
- Thread.sleep(100)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER)
+ store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_SER)
assert(store.getSingle("a2") != None, "a2 was not in store")
assert(store.getSingle("a3") != None, "a3 was not in store")
assert(store.getSingle("a1") === None, "a1 was in store")
assert(store.getSingle("a2") != None, "a2 was not in store")
// At this point a2 was gotten last, so LRU will getSingle rid of a3
- store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_DESER)
- Thread.sleep(100)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER)
assert(store.getSingle("a1") != None, "a1 was not in store")
assert(store.getSingle("a2") != None, "a2 was not in store")
- assert(store.getSingle("a3") === None, "a1 was in store")
+ assert(store.getSingle("a3") === None, "a3 was in store")
}
-
+
+ test("in-memory LRU for partitions of same RDD") {
+ store = new BlockManager(master, serializer, 1200)
+ val a1 = new Array[Byte](400)
+ val a2 = new Array[Byte](400)
+ val a3 = new Array[Byte](400)
+ store.putSingle("rdd_0_1", a1, StorageLevel.MEMORY_ONLY)
+ store.putSingle("rdd_0_2", a2, StorageLevel.MEMORY_ONLY)
+ store.putSingle("rdd_0_3", a3, StorageLevel.MEMORY_ONLY)
+ // Even though we accessed rdd_0_3 last, it should not have replaced partitiosn 1 and 2
+ // from the same RDD
+ assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
+ assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store")
+ assert(store.getSingle("rdd_0_1") != None, "rdd_0_1 was not in store")
+ // Check that rdd_0_3 doesn't replace them even after further accesses
+ assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
+ assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
+ assert(store.getSingle("rdd_0_3") === None, "rdd_0_3 was in store")
+ }
+
+ test("in-memory LRU for partitions of multiple RDDs") {
+ store = new BlockManager(master, serializer, 1200)
+ store.putSingle("rdd_0_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle("rdd_0_2", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle("rdd_1_1", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ // At this point rdd_1_1 should've replaced rdd_0_1
+ assert(store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was not in store")
+ assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store")
+ assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store")
+ // Do a get() on rdd_0_2 so that it is the most recently used item
+ assert(store.getSingle("rdd_0_2") != None, "rdd_0_2 was not in store")
+ // Put in more partitions from RDD 0; they should replace rdd_1_1
+ store.putSingle("rdd_0_3", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ store.putSingle("rdd_0_4", new Array[Byte](400), StorageLevel.MEMORY_ONLY)
+ // Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped
+ // when we try to add rdd_0_4.
+ assert(!store.memoryStore.contains("rdd_1_1"), "rdd_1_1 was in store")
+ assert(!store.memoryStore.contains("rdd_0_1"), "rdd_0_1 was in store")
+ assert(!store.memoryStore.contains("rdd_0_4"), "rdd_0_4 was in store")
+ assert(store.memoryStore.contains("rdd_0_2"), "rdd_0_2 was not in store")
+ assert(store.memoryStore.contains("rdd_0_3"), "rdd_0_3 was not in store")
+ }
+
test("on-disk storage") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
store.putSingle("a1", a1, StorageLevel.DISK_ONLY)
store.putSingle("a2", a2, StorageLevel.DISK_ONLY)
store.putSingle("a3", a3, StorageLevel.DISK_ONLY)
- assert(store.getSingle("a2") != None, "a2 was not in store")
- assert(store.getSingle("a3") != None, "a3 was not in store")
- assert(store.getSingle("a1") != None, "a1 was not in store")
+ assert(store.getSingle("a2") != None, "a2 was in store")
+ assert(store.getSingle("a3") != None, "a3 was in store")
+ assert(store.getSingle("a1") != None, "a1 was in store")
}
test("disk and memory storage") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
- store.putSingle("a1", a1, StorageLevel.DISK_AND_MEMORY_DESER)
- store.putSingle("a2", a2, StorageLevel.DISK_AND_MEMORY_DESER)
- store.putSingle("a3", a3, StorageLevel.DISK_AND_MEMORY_DESER)
- Thread.sleep(100)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK)
+ store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK)
assert(store.getSingle("a2") != None, "a2 was not in store")
assert(store.getSingle("a3") != None, "a3 was not in store")
+ assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store")
assert(store.getSingle("a1") != None, "a1 was not in store")
+ assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store")
+ }
+
+ test("disk and memory storage with getLocalBytes") {
+ store = new BlockManager(master, serializer, 1200)
+ val a1 = new Array[Byte](400)
+ val a2 = new Array[Byte](400)
+ val a3 = new Array[Byte](400)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK)
+ store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK)
+ assert(store.getLocalBytes("a2") != None, "a2 was not in store")
+ assert(store.getLocalBytes("a3") != None, "a3 was not in store")
+ assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store")
+ assert(store.getLocalBytes("a1") != None, "a1 was not in store")
+ assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store")
}
test("disk and memory storage with serialization") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
- store.putSingle("a1", a1, StorageLevel.DISK_AND_MEMORY)
- store.putSingle("a2", a2, StorageLevel.DISK_AND_MEMORY)
- store.putSingle("a3", a3, StorageLevel.DISK_AND_MEMORY)
- Thread.sleep(100)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER)
+ store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER)
assert(store.getSingle("a2") != None, "a2 was not in store")
assert(store.getSingle("a3") != None, "a3 was not in store")
+ assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store")
assert(store.getSingle("a1") != None, "a1 was not in store")
+ assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store")
+ }
+
+ test("disk and memory storage with serialization and getLocalBytes") {
+ store = new BlockManager(master, serializer, 1200)
+ val a1 = new Array[Byte](400)
+ val a2 = new Array[Byte](400)
+ val a3 = new Array[Byte](400)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER)
+ store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER)
+ assert(store.getLocalBytes("a2") != None, "a2 was not in store")
+ assert(store.getLocalBytes("a3") != None, "a3 was not in store")
+ assert(store.memoryStore.getValues("a1") == None, "a1 was in memory store")
+ assert(store.getLocalBytes("a1") != None, "a1 was not in store")
+ assert(store.memoryStore.getValues("a1") != None, "a1 was not in memory store")
}
test("LRU with mixed storage levels") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
val a1 = new Array[Byte](400)
val a2 = new Array[Byte](400)
val a3 = new Array[Byte](400)
val a4 = new Array[Byte](400)
// First store a1 and a2, both in memory, and a3, on disk only
- store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY)
- store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY)
+ store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER)
+ store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER)
store.putSingle("a3", a3, StorageLevel.DISK_ONLY)
// At this point LRU should not kick in because a3 is only on disk
assert(store.getSingle("a1") != None, "a2 was not in store")
@@ -179,8 +256,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
assert(store.getSingle("a2") != None, "a3 was not in store")
assert(store.getSingle("a3") != None, "a1 was not in store")
// Now let's add in a4, which uses both disk and memory; a1 should drop out
- store.putSingle("a4", a4, StorageLevel.DISK_AND_MEMORY)
- Thread.sleep(100)
+ store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER)
assert(store.getSingle("a1") == None, "a1 was in store")
assert(store.getSingle("a2") != None, "a2 was not in store")
assert(store.getSingle("a3") != None, "a3 was not in store")
@@ -188,14 +264,13 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("in-memory LRU with streams") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
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))
- store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_DESER)
- store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_DESER)
- store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY_DESER)
- Thread.sleep(100)
+ store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, true)
+ store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, true)
+ store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, true)
assert(store.get("list2") != None, "list2 was not in store")
assert(store.get("list2").get.size == 2)
assert(store.get("list3") != None, "list3 was not in store")
@@ -204,8 +279,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
assert(store.get("list2") != None, "list2 was not in store")
assert(store.get("list2").get.size == 2)
// At this point list2 was gotten last, so LRU will getSingle rid of list3
- store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_DESER)
- Thread.sleep(100)
+ store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, true)
assert(store.get("list1") != None, "list1 was not in store")
assert(store.get("list1").get.size == 2)
assert(store.get("list2") != None, "list2 was not in store")
@@ -214,16 +288,15 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
}
test("LRU with mixed storage levels and streams") {
- val store = new BlockManager(master, new KryoSerializer, 1200)
+ store = new BlockManager(master, serializer, 1200)
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))
val list4 = List(new Array[Byte](200), new Array[Byte](200))
// First store list1 and list2, both in memory, and list3, on disk only
- store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY)
- store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY)
- store.put("list3", list3.iterator, StorageLevel.DISK_ONLY)
- Thread.sleep(100)
+ store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, true)
+ store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, true)
+ store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, true)
// At this point LRU should not kick in because list3 is only on disk
assert(store.get("list1") != None, "list2 was not in store")
assert(store.get("list1").get.size === 2)
@@ -238,8 +311,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
assert(store.get("list3") != None, "list1 was not in store")
assert(store.get("list3").get.size === 2)
// Now let's add in list4, which uses both disk and memory; list1 should drop out
- store.put("list4", list4.iterator, StorageLevel.DISK_AND_MEMORY)
- Thread.sleep(100)
+ store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, true)
assert(store.get("list1") === None, "list1 was in store")
assert(store.get("list2") != None, "list3 was not in store")
assert(store.get("list2").get.size === 2)
@@ -260,4 +332,70 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT
assert(stream.read() === -1, "end of stream not signalled")
assert(stream.read(temp, 0, temp.length) === -1, "end of stream not signalled")
}
+
+ test("overly large block") {
+ store = new BlockManager(master, serializer, 500)
+ 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)
+ assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store")
+ assert(store.getSingle("a2") != None, "a2 was not in store")
+ }
+
+ test("block compression") {
+ try {
+ System.setProperty("spark.shuffle.compress", "true")
+ store = new BlockManager(master, serializer, 2000)
+ store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize("shuffle_0_0_0") <= 100, "shuffle_0_0_0 was not compressed")
+ store.stop()
+ store = null
+
+ System.setProperty("spark.shuffle.compress", "false")
+ store = new BlockManager(master, serializer, 2000)
+ store.putSingle("shuffle_0_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize("shuffle_0_0_0") >= 1000, "shuffle_0_0_0 was compressed")
+ store.stop()
+ store = null
+
+ System.setProperty("spark.broadcast.compress", "true")
+ store = new BlockManager(master, serializer, 2000)
+ store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize("broadcast_0") <= 100, "broadcast_0 was not compressed")
+ store.stop()
+ store = null
+
+ System.setProperty("spark.broadcast.compress", "false")
+ store = new BlockManager(master, serializer, 2000)
+ store.putSingle("broadcast_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize("broadcast_0") >= 1000, "broadcast_0 was compressed")
+ store.stop()
+ store = null
+
+ System.setProperty("spark.rdd.compress", "true")
+ store = new BlockManager(master, serializer, 2000)
+ store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize("rdd_0_0") <= 100, "rdd_0_0 was not compressed")
+ store.stop()
+ store = null
+
+ System.setProperty("spark.rdd.compress", "false")
+ store = new BlockManager(master, serializer, 2000)
+ store.putSingle("rdd_0_0", new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER)
+ assert(store.memoryStore.getSize("rdd_0_0") >= 1000, "rdd_0_0 was compressed")
+ store.stop()
+ store = null
+
+ // Check that any other block types are also kept uncompressed
+ store = new BlockManager(master, serializer, 2000)
+ store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY)
+ assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed")
+ store.stop()
+ store = null
+ } finally {
+ System.clearProperty("spark.shuffle.compress")
+ System.clearProperty("spark.broadcast.compress")
+ System.clearProperty("spark.rdd.compress")
+ }
+ }
}
diff --git a/docs/README.md b/docs/README.md
new file mode 100644
index 0000000000..092153070e
--- /dev/null
+++ b/docs/README.md
@@ -0,0 +1,34 @@
+Welcome to the Spark documentation!
+
+This readme will walk you through navigating and building the Spark documentation, which is included here with the Spark source code. You can also find documentation specific to release versions of Spark at http://spark-project.org/documentation.html.
+
+Read on to learn more about viewing documentation in plain text (i.e., markdown) or building the documentation yourself. Why build it yourself? So that you have the docs that corresponds to whichever version of Spark you currently have checked out of revision control.
+
+## Generating the Documentation HTML
+
+We include the Spark documentation as part of the source (as opposed to using a hosted wiki, such as the github wiki, as the definitive documentation) to enable the documentation to evolve along with the source code and be captured by revision control (currently git). This way the code automatically includes the version of the documentation that is relevant regardless of which version or release you have checked out or downloaded.
+
+In this directory you will find textfiles formatted using Markdown, with an ".md" suffix. You can read those text files directly if you want. Start with index.md.
+
+To make things quite a bit prettier and make the links easier to follow, generate the html version of the documentation based on the src directory by running `jekyll` in the docs directory. Use the command `SKIP_SCALADOC=1 jekyll` to skip building and copying over the scaladoc which can be timely. To use the `jekyll` command, you will need to have Jekyll installed, the easiest way to do this is via a Ruby Gem, see the [jekyll installation instructions](https://github.com/mojombo/jekyll/wiki/install). This will create a directory called _site containing index.html as well as the rest of the compiled files. Read more about Jekyll at https://github.com/mojombo/jekyll/wiki.
+
+In addition to generating the site as html from the markdown files, jekyll can serve up the site via a webserver. To build and run a webserver use the command `jekyll --server` which (currently) runs the webserver on port 4000, then visit the site at http://localhost:4000.
+
+## Pygments
+
+We also use pygments (http://pygments.org) for syntax highlighting in documentation markdown pages, so you will also need to install that (it requires Python) by running `sudo easy_install Pygments`.
+
+To mark a block of code in your markdown to be syntax highlighted by jekyll during the compile phase, use the following sytax:
+
+ {% highlight scala %}
+ // Your scala code goes here, you can replace scala with many other
+ // supported languages too.
+ {% endhighlight %}
+
+## Scaladoc
+
+You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory.
+
+When you run `jekyll` in the docs directory, it will also copy over the scala doc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc.
+
+NOTE: To skip the step of building and copying over the scaladoc when you build the docs, run `SKIP_SCALADOC=1 jekyll`.
diff --git a/docs/_config.yml b/docs/_config.yml
new file mode 100644
index 0000000000..2bd2eecc86
--- /dev/null
+++ b/docs/_config.yml
@@ -0,0 +1,9 @@
+pygments: true
+markdown: kramdown
+
+# These allow the documentation to be updated with nerw releases
+# of Spark, Scala, and Mesos.
+SPARK_VERSION: 0.7.0-SNAPSHOT
+SPARK_VERSION_SHORT: 0.7.0
+SCALA_VERSION: 2.9.2
+MESOS_VERSION: 0.9.0-incubating
diff --git a/docs/_layouts/404.html b/docs/_layouts/404.html
new file mode 100755
index 0000000000..044654413f
--- /dev/null
+++ b/docs/_layouts/404.html
@@ -0,0 +1,157 @@
+<!DOCTYPE html>
+<html lang="en">
+ <head>
+ <meta charset="utf-8">
+ <title>Page Not Found :(</title>
+ <style>
+ ::-moz-selection {
+ background: #b3d4fc;
+ text-shadow: none;
+ }
+
+ ::selection {
+ background: #b3d4fc;
+ text-shadow: none;
+ }
+
+ html {
+ padding: 30px 10px;
+ font-size: 20px;
+ line-height: 1.4;
+ color: #737373;
+ background: #f0f0f0;
+ -webkit-text-size-adjust: 100%;
+ -ms-text-size-adjust: 100%;
+ }
+
+ html,
+ input {
+ font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
+ }
+
+ body {
+ max-width: 500px;
+ _width: 500px;
+ padding: 30px 20px 50px;
+ border: 1px solid #b3b3b3;
+ border-radius: 4px;
+ margin: 0 auto;
+ box-shadow: 0 1px 10px #a7a7a7, inset 0 1px 0 #fff;
+ background: #fcfcfc;
+ }
+
+ h1 {
+ margin: 0 10px;
+ font-size: 50px;
+ text-align: center;
+ }
+
+ h1 span {
+ color: #bbb;
+ }
+
+ h3 {
+ margin: 1.5em 0 0.5em;
+ }
+
+ p {
+ margin: 1em 0;
+ }
+
+ ul {
+ padding: 0 0 0 40px;
+ margin: 1em 0;
+ }
+
+ .container {
+ max-width: 380px;
+ _width: 380px;
+ margin: 0 auto;
+ }
+
+ /* google search */
+
+ #goog-fixurl ul {
+ list-style: none;
+ padding: 0;
+ margin: 0;
+ }
+
+ #goog-fixurl form {
+ margin: 0;
+ }
+
+ #goog-wm-qt,
+ #goog-wm-sb {
+ border: 1px solid #bbb;
+ font-size: 16px;
+ line-height: normal;
+ vertical-align: top;
+ color: #444;
+ border-radius: 2px;
+ }
+
+ #goog-wm-qt {
+ width: 220px;
+ height: 20px;
+ padding: 5px;
+ margin: 5px 10px 0 0;
+ box-shadow: inset 0 1px 1px #ccc;
+ }
+
+ #goog-wm-sb {
+ display: inline-block;
+ height: 32px;
+ padding: 0 10px;
+ margin: 5px 0 0;
+ white-space: nowrap;
+ cursor: pointer;
+ background-color: #f5f5f5;
+ background-image: -webkit-linear-gradient(rgba(255,255,255,0), #f1f1f1);
+ background-image: -moz-linear-gradient(rgba(255,255,255,0), #f1f1f1);
+ background-image: -ms-linear-gradient(rgba(255,255,255,0), #f1f1f1);
+ background-image: -o-linear-gradient(rgba(255,255,255,0), #f1f1f1);
+ -webkit-appearance: none;
+ -moz-appearance: none;
+ appearance: none;
+ *overflow: visible;
+ *display: inline;
+ *zoom: 1;
+ }
+
+ #goog-wm-sb:hover,
+ #goog-wm-sb:focus {
+ border-color: #aaa;
+ box-shadow: 0 1px 1px rgba(0, 0, 0, 0.1);
+ background-color: #f8f8f8;
+ }
+
+ #goog-wm-qt:hover,
+ #goog-wm-qt:focus {
+ border-color: #105cb6;
+ outline: 0;
+ color: #222;
+ }
+
+ input::-moz-focus-inner {
+ padding: 0;
+ border: 0;
+ }
+ </style>
+ </head>
+ <body>
+ <div class="container">
+ <h1>Not found <span>:(</span></h1>
+ <p>Sorry, but the page you were trying to view does not exist.</p>
+ <p>It looks like this was the result of either:</p>
+ <ul>
+ <li>a mistyped address</li>
+ <li>an out-of-date link</li>
+ </ul>
+ <script>
+ var GOOG_FIXURL_LANG = (navigator.language || '').slice(0,2),GOOG_FIXURL_SITE = location.host;
+ </script>
+ <script src="http://linkhelp.clients.google.com/tbproxy/lh/wm/fixurl.js"></script>
+ </div>
+ </body>
+</html>
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
new file mode 100755
index 0000000000..7244ab6fc9
--- /dev/null
+++ b/docs/_layouts/global.html
@@ -0,0 +1,142 @@
+<!DOCTYPE html>
+<!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]-->
+<!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]-->
+<!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]-->
+<!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]-->
+ <head>
+ <meta charset="utf-8">
+ <meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1">
+ <title>{{ page.title }} - Spark {{site.SPARK_VERSION}} Documentation</title>
+ <meta name="description" content="">
+
+ <link rel="stylesheet" href="css/bootstrap.min.css">
+ <style>
+ body {
+ padding-top: 60px;
+ padding-bottom: 40px;
+ }
+ </style>
+ <meta name="viewport" content="width=device-width">
+ <link rel="stylesheet" href="css/bootstrap-responsive.min.css">
+ <link rel="stylesheet" href="css/main.css">
+
+ <script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
+
+ <link rel="stylesheet" href="css/pygments-default.css">
+ </head>
+ <body>
+ <!--[if lt IE 7]>
+ <p class="chromeframe">You are using an outdated browser. <a href="http://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p>
+ <![endif]-->
+
+ <!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html -->
+
+ <div class="navbar navbar-fixed-top" id="topbar">
+ <div class="navbar-inner">
+ <div class="container">
+ <div class="brand"><a href="index.html">
+ <img src="img/spark-logo-77x50px-hd.png" /></a><span class="version">{{site.SPARK_VERSION_SHORT}}</span>
+ </div>
+ <ul class="nav">
+ <!--TODO(andyk): Add class="active" attribute to li some how.-->
+ <li><a href="index.html">Overview</a></li>
+
+ <li class="dropdown">
+ <a href="#" class="dropdown-toggle" data-toggle="dropdown">Programming Guides<b class="caret"></b></a>
+ <ul class="dropdown-menu">
+ <li><a href="quick-start.html">Quick Start</a></li>
+ <li><a href="scala-programming-guide.html">Scala</a></li>
+ <li><a href="java-programming-guide.html">Java</a></li>
+ </ul>
+ </li>
+
+ <li><a href="api/core/index.html">API (Scaladoc)</a></li>
+
+ <li class="dropdown">
+ <a href="#" class="dropdown-toggle" data-toggle="dropdown">Deploying<b class="caret"></b></a>
+ <ul class="dropdown-menu">
+ <li><a href="ec2-scripts.html">Amazon EC2</a></li>
+ <li><a href="spark-standalone.html">Standalone Mode</a></li>
+ <li><a href="running-on-mesos.html">Mesos</a></li>
+ <li><a href="running-on-yarn.html">YARN</a></li>
+ </ul>
+ </li>
+
+ <li class="dropdown">
+ <a href="api.html" class="dropdown-toggle" data-toggle="dropdown">More<b class="caret"></b></a>
+ <ul class="dropdown-menu">
+ <li><a href="configuration.html">Configuration</a></li>
+ <li><a href="tuning.html">Tuning Guide</a></li>
+ <li><a href="bagel-programming-guide.html">Bagel (Pregel on Spark)</a></li>
+ <li><a href="contributing-to-spark.html">Contributing to Spark</a></li>
+ </ul>
+ </li>
+ </ul>
+ <!--<p class="navbar-text pull-right"><span class="version-text">v{{site.SPARK_VERSION}}</span></p>-->
+ </div>
+ </div>
+ </div>
+
+ <div class="container" id="content">
+ <h1 class="title">{{ page.title }}</h1>
+
+ {{ content }}
+ <!-- Main hero unit for a primary marketing message or call to action -->
+ <!--<div class="hero-unit">
+ <h1>Hello, world!</h1>
+ <p>This is a template for a simple marketing or informational website. It includes a large callout called the hero unit and three supporting pieces of content. Use it as a starting point to create something more unique.</p>
+ <p><a class="btn btn-primary btn-large">Learn more &raquo;</a></p>
+ </div>-->
+
+ <!-- Example row of columns -->
+ <!--<div class="row">
+ <div class="span4">
+ <h2>Heading</h2>
+ <p>Donec id elit non mi porta gravida at eget metus. Fusce dapibus, tellus ac cursus commodo, tortor mauris condimentum nibh, ut fermentum massa justo sit amet risus. Etiam porta sem malesuada magna mollis euismod. Donec sed odio dui. </p>
+ <p><a class="btn" href="#">View details &raquo;</a></p>
+ </div>
+ <div class="span4">
+ <h2>Heading</h2>
+ <p>Donec id elit non mi porta gravida at eget metus. Fusce dapibus, tellus ac cursus commodo, tortor mauris condimentum nibh, ut fermentum massa justo sit amet risus. Etiam porta sem malesuada magna mollis euismod. Donec sed odio dui. </p>
+ <p><a class="btn" href="#">View details &raquo;</a></p>
+ </div>
+ <div class="span4">
+ <h2>Heading</h2>
+ <p>Donec sed odio dui. Cras justo odio, dapibus ac facilisis in, egestas eget quam. Vestibulum id ligula porta felis euismod semper. Fusce dapibus, tellus ac cursus commodo, tortor mauris condimentum nibh, ut fermentum massa justo sit amet risus.</p>
+ <p><a class="btn" href="#">View details &raquo;</a></p>
+ </div>
+ </div>
+
+ <hr>-->
+
+ <!--<footer>
+ <p></p>
+ </footer>-->
+
+ </div> <!-- /container -->
+
+ <script src="js/vendor/jquery-1.8.0.min.js"></script>
+ <script src="js/vendor/bootstrap.min.js"></script>
+ <script src="js/main.js"></script>
+
+ <!-- A script to fix internal hash links because we have an overlapping top bar.
+ Based on https://github.com/twitter/bootstrap/issues/193#issuecomment-2281510 -->
+ <script>
+ $(function() {
+ function maybeScrollToHash() {
+ if (window.location.hash && $(window.location.hash).length) {
+ var newTop = $(window.location.hash).offset().top - $('#topbar').height() - 5;
+ $(window).scrollTop(newTop);
+ }
+ }
+ $(window).bind('hashchange', function() {
+ maybeScrollToHash();
+ });
+ // Scroll now too in case we had opened the page on a hash, but wait 1 ms because some browsers
+ // will try to do *their* initial scroll after running the onReady handler.
+ setTimeout(function() { maybeScrollToHash(); }, 1)
+ })
+ </script>
+
+ </body>
+</html>
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
new file mode 100644
index 0000000000..e61c105449
--- /dev/null
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -0,0 +1,30 @@
+require 'fileutils'
+include FileUtils
+
+if ENV['SKIP_SCALADOC'] != '1'
+ projects = ["core", "examples", "repl", "bagel"]
+
+ puts "Moving to project root and building scaladoc."
+ curr_dir = pwd
+ cd("..")
+
+ puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..."
+ puts `sbt/sbt doc`
+
+ puts "moving back into docs dir."
+ cd("docs")
+
+ # 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.2/api"
+ dest = "api/" + project_name
+
+ puts "echo making directory " + dest
+ mkdir_p dest
+
+ # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't.
+ puts "cp -r " + source + "/. " + dest
+ cp_r(source + "/.", dest)
+ end
+end
diff --git a/docs/api.md b/docs/api.md
new file mode 100644
index 0000000000..43548b223c
--- /dev/null
+++ b/docs/api.md
@@ -0,0 +1,10 @@
+---
+layout: global
+title: Spark API documentation (Scaladoc)
+---
+
+Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory.
+
+- [Core](api/core/index.html)
+- [Examples](api/examples/index.html)
+- [Bagel](api/bagel/index.html)
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
new file mode 100644
index 0000000000..8a0fa42d94
--- /dev/null
+++ b/docs/bagel-programming-guide.md
@@ -0,0 +1,161 @@
+---
+layout: global
+title: Bagel Programming Guide
+---
+
+**Bagel** is a Spark implementation of Google's [Pregel](http://portal.acm.org/citation.cfm?id=1807184) graph processing framework. Bagel currently supports basic graph computation, combiners, and aggregators.
+
+In the Pregel programming model, jobs run as a sequence of iterations called _supersteps_. In each superstep, each vertex in the graph runs a user-specified function that can update state associated with the vertex and send messages to other vertices for use in the *next* iteration.
+
+This guide shows the programming model and features of Bagel by walking through an example implementation of PageRank on Bagel.
+
+## Linking with Bagel
+
+To write a Bagel application, you will need to add Spark, its dependencies, and Bagel to your CLASSPATH:
+
+1. Run `sbt/sbt update` to fetch Spark's dependencies, if you haven't already done so.
+2. Run `sbt/sbt assembly` to build Spark and its dependencies into one JAR (`core/target/spark-core-assembly-{{site.SPARK_VERSION}}.jar`)
+3. Run `sbt/sbt package` build the Bagel JAR (`bagel/target/scala_{{site.SCALA_VERSION}}/spark-bagel_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar`).
+4. Add these two JARs to your CLASSPATH.
+
+## Programming Model
+
+Bagel operates on a graph represented as a [distributed dataset](scala-programming-guide.html) of (K, V) pairs, where keys are vertex IDs and values are vertices plus their associated state. In each superstep, Bagel runs a user-specified compute function on each vertex that takes as input the current vertex state and a list of messages sent to that vertex during the previous superstep, and returns the new vertex state and a list of outgoing messages.
+
+For example, we can use Bagel to implement PageRank. Here, vertices represent pages, edges represent links between pages, and messages represent shares of PageRank sent to the pages that a particular page links to.
+
+We first extend the default `Vertex` class to store a `Double`
+representing the current PageRank of the vertex, and similarly extend
+the `Message` and `Edge` classes. Note that these need to be marked `@serializable` to allow Spark to transfer them across machines. We also import the Bagel types and implicit conversions.
+
+{% highlight scala %}
+import spark.bagel._
+import spark.bagel.Bagel._
+
+@serializable class PREdge(val targetId: String) extends Edge
+
+@serializable class PRVertex(
+ val id: String, val rank: Double, val outEdges: Seq[Edge],
+ val active: Boolean) extends Vertex
+
+@serializable class PRMessage(
+ val targetId: String, val rankShare: Double) extends Message
+{% endhighlight %}
+
+Next, we load a sample graph from a text file as a distributed dataset and package it into `PRVertex` objects. We also cache the distributed dataset because Bagel will use it multiple times and we'd like to avoid recomputing it.
+
+{% highlight scala %}
+val input = sc.textFile("pagerank_data.txt")
+
+val numVerts = input.count()
+
+val verts = input.map(line => {
+ val fields = line.split('\t')
+ val (id, linksStr) = (fields(0), fields(1))
+ val links = linksStr.split(',').map(new PREdge(_))
+ (id, new PRVertex(id, 1.0 / numVerts, links, true))
+}).cache
+{% endhighlight %}
+
+We run the Bagel job, passing in `verts`, an empty distributed dataset of messages, and a custom compute function that runs PageRank for 10 iterations.
+
+{% highlight scala %}
+val emptyMsgs = sc.parallelize(List[(String, PRMessage)]())
+
+def compute(self: PRVertex, msgs: Option[Seq[PRMessage]], superstep: Int)
+: (PRVertex, Iterable[PRMessage]) = {
+ val msgSum = msgs.getOrElse(List()).map(_.rankShare).sum
+ val newRank =
+ if (msgSum != 0)
+ 0.15 / numVerts + 0.85 * msgSum
+ else
+ self.rank
+ val halt = superstep >= 10
+ val msgsOut =
+ if (!halt)
+ self.outEdges.map(edge =>
+ new PRMessage(edge.targetId, newRank / self.outEdges.size))
+ else
+ List()
+ (new PRVertex(self.id, newRank, self.outEdges, !halt), msgsOut)
+}
+{% endhighlight %}
+
+val result = Bagel.run(sc, verts, emptyMsgs)()(compute)
+
+Finally, we print the results.
+
+{% highlight scala %}
+println(result.map(v => "%s\t%s\n".format(v.id, v.rank)).collect.mkString)
+{% endhighlight %}
+
+### Combiners
+
+Sending a message to another vertex generally involves expensive communication over the network. For certain algorithms, it's possible to reduce the amount of communication using _combiners_. For example, if the compute function receives integer messages and only uses their sum, it's possible for Bagel to combine multiple messages to the same vertex by summing them.
+
+For combiner support, Bagel can optionally take a set of combiner functions that convert messages to their combined form.
+
+_Example: PageRank with combiners_
+
+### Aggregators
+
+Aggregators perform a reduce across all vertices after each superstep, and provide the result to each vertex in the next superstep.
+
+For aggregator support, Bagel can optionally take an aggregator function that reduces across each vertex.
+
+_Example_
+
+### Operations
+
+Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/mesos/spark/blob/master/bagel/src/main/scala/spark/bagel/Bagel.scala) for details.
+
+#### Actions
+
+{% highlight scala %}
+/*** Full form ***/
+
+Bagel.run(sc, vertices, messages, combiner, aggregator, partitioner, numSplits)(compute)
+// where compute takes (vertex: V, combinedMessages: Option[C], aggregated: Option[A], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+
+/*** Abbreviated forms ***/
+
+Bagel.run(sc, vertices, messages, combiner, partitioner, numSplits)(compute)
+// where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+
+Bagel.run(sc, vertices, messages, combiner, numSplits)(compute)
+// where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+
+Bagel.run(sc, vertices, messages, numSplits)(compute)
+// where compute takes (vertex: V, messages: Option[Array[M]], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+{% endhighlight %}
+
+#### Types
+
+{% highlight scala %}
+trait Combiner[M, C] {
+ def createCombiner(msg: M): C
+ def mergeMsg(combiner: C, msg: M): C
+ def mergeCombiners(a: C, b: C): C
+}
+
+trait Aggregator[V, A] {
+ def createAggregator(vert: V): A
+ def mergeAggregators(a: A, b: A): A
+}
+
+trait Vertex {
+ def active: Boolean
+}
+
+trait Message[K] {
+ def targetId: K
+}
+{% endhighlight %}
+
+## Where to Go from Here
+
+Two example jobs, PageRank and shortest path, are included in `bagel/src/main/scala/spark/bagel/examples`. You can run them by passing the class name to the `run` script included in Spark -- for example, `./run spark.bagel.examples.WikipediaPageRank`. Each example program prints usage help when run without any arguments.
diff --git a/docs/configuration.md b/docs/configuration.md
new file mode 100644
index 0000000000..d8317ea97c
--- /dev/null
+++ b/docs/configuration.md
@@ -0,0 +1,227 @@
+---
+layout: global
+title: Spark Configuration
+---
+
+Spark provides three main locations to configure the system:
+
+* [Environment variables](#environment-variables) for launching Spark workers, which can
+ be set either in your driver program or in the `conf/spark-env.sh` script.
+* [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 the
+ `SPARK_JAVA_OPTS` environment variable in `spark-env.sh`.
+* [Logging configuration](#configuring-logging), which is done through `log4j.properties`.
+
+
+# Environment Variables
+
+Spark determines how to initialize the JVM on worker nodes, or even on the local node when you run `spark-shell`,
+by running the `conf/spark-env.sh` script in the directory where it is installed. This script does not exist by default
+in the Git repository, but but you can create it by copying `conf/spark-env.sh.template`. Make sure that you make
+the copy executable.
+
+Inside `spark-env.sh`, you *must* set at least the following two variables:
+
+* `SCALA_HOME`, to point to your Scala installation.
+* `MESOS_NATIVE_LIBRARY`, if you are [running on a Mesos cluster](running-on-mesos.html).
+
+In addition, there are four other variables that control execution. These can be set *either in `spark-env.sh`
+or in each job's driver program*, because they will automatically be propagated to workers from the driver.
+For a multi-user environment, we recommend setting the in the driver program instead of `spark-env.sh`, so
+that different user jobs can use different amounts of memory, JVM options, etc.
+
+* `SPARK_MEM`, to set the amount of memory used per node (this should be in the same format as the
+ JVM's -Xmx option, e.g. `300m` or `1g`)
+* `SPARK_JAVA_OPTS`, to add JVM options. This includes any system properties that you'd like to pass with `-D`.
+* `SPARK_CLASSPATH`, to add elements to Spark's classpath.
+* `SPARK_LIBRARY_PATH`, to add search directories for native libraries.
+
+Note that if you do set these in `spark-env.sh`, they will override the values set by user programs, which
+is undesirable; you can choose to have `spark-env.sh` set them only if the user program hasn't, as follows:
+
+{% highlight bash %}
+if [ -z "$SPARK_MEM" ] ; then
+ SPARK_MEM="1g"
+fi
+{% endhighlight %}
+
+# System 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:
+
+{% highlight scala %}
+System.setProperty("spark.cores.max", "5")
+val sc = new SparkContext(...)
+{% endhighlight %}
+
+Most of the configurable system properties control internal settings that have reasonable default values. However,
+there are at least four properties that you will commonly want to control:
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+ <td>spark.serializer</td>
+ <td>spark.JavaSerializer</td>
+ <td>
+ Class to use for serializing objects that will be sent over the network or need to be cached
+ in serialized form. The default of Java serialization works with any Serializable Java object but is
+ quite slow, so we recommend <a href="tuning.html">using <code>spark.KryoSerializer</code>
+ and configuring Kryo serialization</a> when speed is necessary. Can be any subclass of
+ <a href="api/core/index.html#spark.Serializer"><code>spark.Serializer</code></a>).
+ </td>
+</tr>
+<tr>
+ <td>spark.kryo.registrator</td>
+ <td>(none)</td>
+ <td>
+ If you use Kryo serialization, set this class to register your custom classes with Kryo.
+ You need to set it to a class that extends
+ <a href="api/core/index.html#spark.KryoRegistrator"><code>spark.KryoRegistrator</code></a>).
+ See the <a href="tuning.html#data-serialization">tuning guide</a> for more details.
+ </td>
+</tr>
+<tr>
+ <td>spark.local.dir</td>
+ <td>/tmp</td>
+ <td>
+ Directory to use for "scratch" space in Spark, including map output files and RDDs that get stored
+ on disk. This should be on a fast, local disk in your system. It can also be a comma-separated
+ list of multiple directories.
+ </td>
+</tr>
+<tr>
+ <td>spark.cores.max</td>
+ <td>(infinite)</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.
+ </td>
+</tr>
+</table>
+
+
+Apart from these, the following properties are also available, and may be useful in some situations:
+
+<table class="table">
+<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
+<tr>
+ <td>spark.mesos.coarse</td>
+ <td>false</td>
+ <td>
+ If set to "true", runs over Mesos clusters in
+ <a href="running-on-mesos.html#mesos-run-modes">"coarse-grained" sharing mode</a>,
+ where Spark acquires one long-lived Mesos task on each machine instead of one Mesos task per Spark task.
+ This gives lower-latency scheduling for short queries, but leaves resources in use for the whole
+ duration of the Spark job.
+ </td>
+</tr>
+<tr>
+ <td>spark.default.parallelism</td>
+ <td>8</td>
+ <td>
+ Default number of tasks to use for distributed shuffle operations (<code>groupByKey</code>,
+ <code>reduceByKey</code>, etc) when not set by user.
+ </td>
+</tr>
+<tr>
+ <td>spark.storage.memoryFraction</td>
+ <td>0.66</td>
+ <td>
+ Fraction of Java heap to use for Spark's memory cache. This should not be larger than the "old"
+ generation of objects in the JVM, which by default is given 2/3 of the heap, but you can increase
+ it if you configure your own old generation size.
+ </td>
+</tr>
+<tr>
+ <td>spark.shuffle.compress</td>
+ <td>true</td>
+ <td>
+ Whether to compress map output files. Generally a good idea.
+ </td>
+</tr>
+<tr>
+ <td>spark.broadcast.compress</td>
+ <td>true</td>
+ <td>
+ Whether to compress broadcast variables before sending them. Generally a good idea.
+ </td>
+</tr>
+<tr>
+ <td>spark.rdd.compress</td>
+ <td>false</td>
+ <td>
+ Whether to compress serialized RDD partitions (e.g. for <code>StorageLevel.MEMORY_ONLY_SER</code>).
+ Can save substantial space at the cost of some extra CPU time.
+ </td>
+</tr>
+<tr>
+ <td>spark.reducer.maxMbInFlight</td>
+ <td>48</td>
+ <td>
+ Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since
+ each output requires us to create a buffer to receive it, this represents a fixed memory overhead
+ per reduce task, so keep it small unless you have a large amount of memory.
+ </td>
+</tr>
+<tr>
+ <td>spark.closure.serializer</td>
+ <td>spark.JavaSerializer</td>
+ <td>
+ Serializer class to use for closures. Generally Java is fine unless your distributed functions
+ (e.g. map functions) reference large objects in the driver program.
+ </td>
+</tr>
+<tr>
+ <td>spark.kryoserializer.buffer.mb</td>
+ <td>32</td>
+ <td>
+ Maximum object size to allow within Kryo (the library needs to create a buffer at least as
+ large as the largest single object you'll serialize). Increase this if you get a "buffer limit
+ exceeded" exception inside Kryo. Note that there will be one buffer <i>per core</i> on each worker.
+ </td>
+</tr>
+<tr>
+ <td>spark.broadcast.factory</td>
+ <td>spark.broadcast. HttpBroadcastFactory</td>
+ <td>
+ Which broadcast implementation to use.
+ </td>
+</tr>
+<tr>
+ <td>spark.locality.wait</td>
+ <td>3000</td>
+ <td>
+ Number of milliseconds to wait to launch a data-local task before giving up and launching it
+ in a non-data-local location. You should increase this if your tasks are long and you are seeing
+ poor data locality, but the default generally works well.
+ </td>
+</tr>
+<tr>
+ <td>spark.akka.threads</td>
+ <td>4</td>
+ <td>
+ Number of actor threads to use for communication. Can be useful to increase on large clusters
+ when the master has a lot of CPU cores.
+ </td>
+</tr>
+<tr>
+ <td>spark.master.host</td>
+ <td>(local hostname)</td>
+ <td>
+ Hostname or IP address for the master to listen on.
+ </td>
+</tr>
+<tr>
+ <td>spark.master.port</td>
+ <td>(random)</td>
+ <td>
+ Port for the master to listen on.
+ </td>
+</tr>
+</table>
+
+# Configuring Logging
+
+Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j.properties`
+file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there.
diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md
new file mode 100644
index 0000000000..c6e01c62d8
--- /dev/null
+++ b/docs/contributing-to-spark.md
@@ -0,0 +1,26 @@
+---
+layout: global
+title: Contributing to Spark
+---
+
+The Spark team welcomes contributions in the form of GitHub pull requests. Here are a few tips to get your contribution in:
+
+- Break your work into small, single-purpose patches if possible. It's much harder to merge in a large change with a lot of disjoint features.
+- Submit the patch as a GitHub pull request. For a tutorial, see the GitHub guides on [forking a repo](https://help.github.com/articles/fork-a-repo) and [sending a pull request](https://help.github.com/articles/using-pull-requests).
+- Follow the style of the existing codebase. Specifically, we use [standard Scala style guide](http://docs.scala-lang.org/style/), but with the following changes:
+ * Maximum line length of 100 characters.
+ * Always import packages using absolute paths (e.g. `scala.collection.Map` instead of `collection.Map`).
+ * No "infix" syntax for methods other than operators. For example, don't write `table containsKey myKey`; replace it with `table.containsKey(myKey)`.
+- Make sure that your code passes the unit tests. You can run the tests with `sbt/sbt test` in the root directory of Spark.
+ But first, make sure that you have [configured a spark-env.sh](configuration.html) with at least
+ `SCALA_HOME`, as some of the tests try to spawn subprocesses using this.
+- Add new unit tests for your code. We use [ScalaTest](http://www.scalatest.org/) for testing. Just add a new Suite in `core/src/test`, or methods to an existing Suite.
+- If you'd like to report a bug but don't have time to fix it, you can still post it to our [issues page](https://github.com/mesos/spark/issues), or email the [mailing list](http://www.spark-project.org/mailing-lists.html).
+
+# Licensing of Contributions
+
+Contributions via GitHub pull requests are gladly accepted from their original author. Along with any pull requests, please
+state that the contribution is your original work and that you license the work to the project under the project's open source
+license. *Whether or not you state this explicitly, by submitting any copyrighted material via pull request, email, or other
+means you agree to license the material under the project's open source license and warrant that you have the legal authority
+to do so.*
diff --git a/docs/css/bootstrap-responsive.css b/docs/css/bootstrap-responsive.css
new file mode 100644
index 0000000000..daafa91808
--- /dev/null
+++ b/docs/css/bootstrap-responsive.css
@@ -0,0 +1,1040 @@
+/*!
+ * Bootstrap Responsive v2.1.0
+ *
+ * Copyright 2012 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Designed and built with all the love in the world @twitter by @mdo and @fat.
+ */
+
+.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;
+}
+
+.hidden {
+ display: none;
+ visibility: hidden;
+}
+
+.visible-phone {
+ display: none !important;
+}
+
+.visible-tablet {
+ display: none !important;
+}
+
+.hidden-desktop {
+ display: none !important;
+}
+
+.visible-desktop {
+ display: inherit !important;
+}
+
+@media (min-width: 768px) and (max-width: 979px) {
+ .hidden-desktop {
+ display: inherit !important;
+ }
+ .visible-desktop {
+ display: none !important ;
+ }
+ .visible-tablet {
+ display: inherit !important;
+ }
+ .hidden-tablet {
+ display: none !important;
+ }
+}
+
+@media (max-width: 767px) {
+ .hidden-desktop {
+ display: inherit !important;
+ }
+ .visible-desktop {
+ display: none !important;
+ }
+ .visible-phone {
+ display: inherit !important;
+ }
+ .hidden-phone {
+ display: none !important;
+ }
+}
+
+@media (min-width: 1200px) {
+ .row {
+ margin-left: -30px;
+ *zoom: 1;
+ }
+ .row:before,
+ .row:after {
+ display: table;
+ line-height: 0;
+ content: "";
+ }
+ .row:after {
+ clear: both;
+ }
+ [class*="span"] {
+ float: left;
+ margin-left: 30px;
+ }
+ .container,
+ .navbar-static-top .container,
+ .navbar-fixed-top .container,
+ .navbar-fixed-bottom .container {
+ width: 1170px;
+ }
+ .span12 {
+ width: 1170px;
+ }
+ .span11 {
+ width: 1070px;
+ }
+ .span10 {
+ width: 970px;
+ }
+ .span9 {
+ width: 870px;
+ }
+ .span8 {
+ width: 770px;
+ }
+ .span7 {
+ width: 670px;
+ }
+ .span6 {
+ width: 570px;
+ }
+ .span5 {
+ width: 470px;
+ }
+ .span4 {
+ width: 370px;
+ }
+ .span3 {
+ width: 270px;
+ }
+ .span2 {
+ width: 170px;
+ }
+ .span1 {
+ width: 70px;
+ }
+ .offset12 {
+ margin-left: 1230px;
+ }
+ .offset11 {
+ margin-left: 1130px;
+ }
+ .offset10 {
+ margin-left: 1030px;
+ }
+ .offset9 {
+ margin-left: 930px;
+ }
+ .offset8 {
+ margin-left: 830px;
+ }
+ .offset7 {
+ margin-left: 730px;
+ }
+ .offset6 {
+ margin-left: 630px;
+ }
+ .offset5 {
+ margin-left: 530px;
+ }
+ .offset4 {
+ margin-left: 430px;
+ }
+ .offset3 {
+ margin-left: 330px;
+ }
+ .offset2 {
+ margin-left: 230px;
+ }
+ .offset1 {
+ margin-left: 130px;
+ }
+ .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.564102564102564%;
+ *margin-left: 2.5109110747408616%;
+ -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.45299145299145%;
+ *width: 91.39979996362975%;
+ }
+ .row-fluid .span10 {
+ width: 82.90598290598291%;
+ *width: 82.8527914166212%;
+ }
+ .row-fluid .span9 {
+ width: 74.35897435897436%;
+ *width: 74.30578286961266%;
+ }
+ .row-fluid .span8 {
+ width: 65.81196581196582%;
+ *width: 65.75877432260411%;
+ }
+ .row-fluid .span7 {
+ width: 57.26495726495726%;
+ *width: 57.21176577559556%;
+ }
+ .row-fluid .span6 {
+ width: 48.717948717948715%;
+ *width: 48.664757228587014%;
+ }
+ .row-fluid .span5 {
+ width: 40.17094017094017%;
+ *width: 40.11774868157847%;
+ }
+ .row-fluid .span4 {
+ width: 31.623931623931625%;
+ *width: 31.570740134569924%;
+ }
+ .row-fluid .span3 {
+ width: 23.076923076923077%;
+ *width: 23.023731587561375%;
+ }
+ .row-fluid .span2 {
+ width: 14.52991452991453%;
+ *width: 14.476723040552828%;
+ }
+ .row-fluid .span1 {
+ width: 5.982905982905983%;
+ *width: 5.929714493544281%;
+ }
+ .row-fluid .offset12 {
+ margin-left: 105.12820512820512%;
+ *margin-left: 105.02182214948171%;
+ }
+ .row-fluid .offset12:first-child {
+ margin-left: 102.56410256410257%;
+ *margin-left: 102.45771958537915%;
+ }
+ .row-fluid .offset11 {
+ margin-left: 96.58119658119658%;
+ *margin-left: 96.47481360247316%;
+ }
+ .row-fluid .offset11:first-child {
+ margin-left: 94.01709401709402%;
+ *margin-left: 93.91071103837061%;
+ }
+ .row-fluid .offset10 {
+ margin-left: 88.03418803418803%;
+ *margin-left: 87.92780505546462%;
+ }
+ .row-fluid .offset10:first-child {
+ margin-left: 85.47008547008548%;
+ *margin-left: 85.36370249136206%;
+ }
+ .row-fluid .offset9 {
+ margin-left: 79.48717948717949%;
+ *margin-left: 79.38079650845607%;
+ }
+ .row-fluid .offset9:first-child {
+ margin-left: 76.92307692307693%;
+ *margin-left: 76.81669394435352%;
+ }
+ .row-fluid .offset8 {
+ margin-left: 70.94017094017094%;
+ *margin-left: 70.83378796144753%;
+ }
+ .row-fluid .offset8:first-child {
+ margin-left: 68.37606837606839%;
+ *margin-left: 68.26968539734497%;
+ }
+ .row-fluid .offset7 {
+ margin-left: 62.393162393162385%;
+ *margin-left: 62.28677941443899%;
+ }
+ .row-fluid .offset7:first-child {
+ margin-left: 59.82905982905982%;
+ *margin-left: 59.72267685033642%;
+ }
+ .row-fluid .offset6 {
+ margin-left: 53.84615384615384%;
+ *margin-left: 53.739770867430444%;
+ }
+ .row-fluid .offset6:first-child {
+ margin-left: 51.28205128205128%;
+ *margin-left: 51.175668303327875%;
+ }
+ .row-fluid .offset5 {
+ margin-left: 45.299145299145295%;
+ *margin-left: 45.1927623204219%;
+ }
+ .row-fluid .offset5:first-child {
+ margin-left: 42.73504273504273%;
+ *margin-left: 42.62865975631933%;
+ }
+ .row-fluid .offset4 {
+ margin-left: 36.75213675213675%;
+ *margin-left: 36.645753773413354%;
+ }
+ .row-fluid .offset4:first-child {
+ margin-left: 34.18803418803419%;
+ *margin-left: 34.081651209310785%;
+ }
+ .row-fluid .offset3 {
+ margin-left: 28.205128205128204%;
+ *margin-left: 28.0987452264048%;
+ }
+ .row-fluid .offset3:first-child {
+ margin-left: 25.641025641025642%;
+ *margin-left: 25.53464266230224%;
+ }
+ .row-fluid .offset2 {
+ margin-left: 19.65811965811966%;
+ *margin-left: 19.551736679396257%;
+ }
+ .row-fluid .offset2:first-child {
+ margin-left: 17.094017094017094%;
+ *margin-left: 16.98763411529369%;
+ }
+ .row-fluid .offset1 {
+ margin-left: 11.11111111111111%;
+ *margin-left: 11.004728132387708%;
+ }
+ .row-fluid .offset1:first-child {
+ margin-left: 8.547008547008547%;
+ *margin-left: 8.440625568285142%;
+ }
+ input,
+ textarea,
+ .uneditable-input {
+ margin-left: 0;
+ }
+ .controls-row [class*="span"] + [class*="span"] {
+ margin-left: 30px;
+ }
+ input.span12,
+ textarea.span12,
+ .uneditable-input.span12 {
+ width: 1156px;
+ }
+ input.span11,
+ textarea.span11,
+ .uneditable-input.span11 {
+ width: 1056px;
+ }
+ input.span10,
+ textarea.span10,
+ .uneditable-input.span10 {
+ width: 956px;
+ }
+ input.span9,
+ textarea.span9,
+ .uneditable-input.span9 {
+ width: 856px;
+ }
+ input.span8,
+ textarea.span8,
+ .uneditable-input.span8 {
+ width: 756px;
+ }
+ input.span7,
+ textarea.span7,
+ .uneditable-input.span7 {
+ width: 656px;
+ }
+ input.span6,
+ textarea.span6,
+ .uneditable-input.span6 {
+ width: 556px;
+ }
+ input.span5,
+ textarea.span5,
+ .uneditable-input.span5 {
+ width: 456px;
+ }
+ input.span4,
+ textarea.span4,
+ .uneditable-input.span4 {
+ width: 356px;
+ }
+ input.span3,
+ textarea.span3,
+ .uneditable-input.span3 {
+ width: 256px;
+ }
+ input.span2,
+ textarea.span2,
+ .uneditable-input.span2 {
+ width: 156px;
+ }
+ input.span1,
+ textarea.span1,
+ .uneditable-input.span1 {
+ width: 56px;
+ }
+ .thumbnails {
+ margin-left: -30px;
+ }
+ .thumbnails > li {
+ margin-left: 30px;
+ }
+ .row-fluid .thumbnails {
+ margin-left: 0;
+ }
+}
+
+@media (min-width: 768px) and (max-width: 979px) {
+ .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: 724px;
+ }
+ .span12 {
+ width: 724px;
+ }
+ .span11 {
+ width: 662px;
+ }
+ .span10 {
+ width: 600px;
+ }
+ .span9 {
+ width: 538px;
+ }
+ .span8 {
+ width: 476px;
+ }
+ .span7 {
+ width: 414px;
+ }
+ .span6 {
+ width: 352px;
+ }
+ .span5 {
+ width: 290px;
+ }
+ .span4 {
+ width: 228px;
+ }
+ .span3 {
+ width: 166px;
+ }
+ .span2 {
+ width: 104px;
+ }
+ .span1 {
+ width: 42px;
+ }
+ .offset12 {
+ margin-left: 764px;
+ }
+ .offset11 {
+ margin-left: 702px;
+ }
+ .offset10 {
+ margin-left: 640px;
+ }
+ .offset9 {
+ margin-left: 578px;
+ }
+ .offset8 {
+ margin-left: 516px;
+ }
+ .offset7 {
+ margin-left: 454px;
+ }
+ .offset6 {
+ margin-left: 392px;
+ }
+ .offset5 {
+ margin-left: 330px;
+ }
+ .offset4 {
+ margin-left: 268px;
+ }
+ .offset3 {
+ margin-left: 206px;
+ }
+ .offset2 {
+ margin-left: 144px;
+ }
+ .offset1 {
+ margin-left: 82px;
+ }
+ .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.7624309392265194%;
+ *margin-left: 2.709239449864817%;
+ -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.43646408839778%;
+ *width: 91.38327259903608%;
+ }
+ .row-fluid .span10 {
+ width: 82.87292817679558%;
+ *width: 82.81973668743387%;
+ }
+ .row-fluid .span9 {
+ width: 74.30939226519337%;
+ *width: 74.25620077583166%;
+ }
+ .row-fluid .span8 {
+ width: 65.74585635359117%;
+ *width: 65.69266486422946%;
+ }
+ .row-fluid .span7 {
+ width: 57.18232044198895%;
+ *width: 57.12912895262725%;
+ }
+ .row-fluid .span6 {
+ width: 48.61878453038674%;
+ *width: 48.56559304102504%;
+ }
+ .row-fluid .span5 {
+ width: 40.05524861878453%;
+ *width: 40.00205712942283%;
+ }
+ .row-fluid .span4 {
+ width: 31.491712707182323%;
+ *width: 31.43852121782062%;
+ }
+ .row-fluid .span3 {
+ width: 22.92817679558011%;
+ *width: 22.87498530621841%;
+ }
+ .row-fluid .span2 {
+ width: 14.3646408839779%;
+ *width: 14.311449394616199%;
+ }
+ .row-fluid .span1 {
+ width: 5.801104972375691%;
+ *width: 5.747913483013988%;
+ }
+ .row-fluid .offset12 {
+ margin-left: 105.52486187845304%;
+ *margin-left: 105.41847889972962%;
+ }
+ .row-fluid .offset12:first-child {
+ margin-left: 102.76243093922652%;
+ *margin-left: 102.6560479605031%;
+ }
+ .row-fluid .offset11 {
+ margin-left: 96.96132596685082%;
+ *margin-left: 96.8549429881274%;
+ }
+ .row-fluid .offset11:first-child {
+ margin-left: 94.1988950276243%;
+ *margin-left: 94.09251204890089%;
+ }
+ .row-fluid .offset10 {
+ margin-left: 88.39779005524862%;
+ *margin-left: 88.2914070765252%;
+ }
+ .row-fluid .offset10:first-child {
+ margin-left: 85.6353591160221%;
+ *margin-left: 85.52897613729868%;
+ }
+ .row-fluid .offset9 {
+ margin-left: 79.8342541436464%;
+ *margin-left: 79.72787116492299%;
+ }
+ .row-fluid .offset9:first-child {
+ margin-left: 77.07182320441989%;
+ *margin-left: 76.96544022569647%;
+ }
+ .row-fluid .offset8 {
+ margin-left: 71.2707182320442%;
+ *margin-left: 71.16433525332079%;
+ }
+ .row-fluid .offset8:first-child {
+ margin-left: 68.50828729281768%;
+ *margin-left: 68.40190431409427%;
+ }
+ .row-fluid .offset7 {
+ margin-left: 62.70718232044199%;
+ *margin-left: 62.600799341718584%;
+ }
+ .row-fluid .offset7:first-child {
+ margin-left: 59.94475138121547%;
+ *margin-left: 59.838368402492065%;
+ }
+ .row-fluid .offset6 {
+ margin-left: 54.14364640883978%;
+ *margin-left: 54.037263430116376%;
+ }
+ .row-fluid .offset6:first-child {
+ margin-left: 51.38121546961326%;
+ *margin-left: 51.27483249088986%;
+ }
+ .row-fluid .offset5 {
+ margin-left: 45.58011049723757%;
+ *margin-left: 45.47372751851417%;
+ }
+ .row-fluid .offset5:first-child {
+ margin-left: 42.81767955801105%;
+ *margin-left: 42.71129657928765%;
+ }
+ .row-fluid .offset4 {
+ margin-left: 37.01657458563536%;
+ *margin-left: 36.91019160691196%;
+ }
+ .row-fluid .offset4:first-child {
+ margin-left: 34.25414364640884%;
+ *margin-left: 34.14776066768544%;
+ }
+ .row-fluid .offset3 {
+ margin-left: 28.45303867403315%;
+ *margin-left: 28.346655695309746%;
+ }
+ .row-fluid .offset3:first-child {
+ margin-left: 25.69060773480663%;
+ *margin-left: 25.584224756083227%;
+ }
+ .row-fluid .offset2 {
+ margin-left: 19.88950276243094%;
+ *margin-left: 19.783119783707537%;
+ }
+ .row-fluid .offset2:first-child {
+ margin-left: 17.12707182320442%;
+ *margin-left: 17.02068884448102%;
+ }
+ .row-fluid .offset1 {
+ margin-left: 11.32596685082873%;
+ *margin-left: 11.219583872105325%;
+ }
+ .row-fluid .offset1:first-child {
+ margin-left: 8.56353591160221%;
+ *margin-left: 8.457152932878806%;
+ }
+ input,
+ textarea,
+ .uneditable-input {
+ margin-left: 0;
+ }
+ .controls-row [class*="span"] + [class*="span"] {
+ margin-left: 20px;
+ }
+ input.span12,
+ textarea.span12,
+ .uneditable-input.span12 {
+ width: 710px;
+ }
+ input.span11,
+ textarea.span11,
+ .uneditable-input.span11 {
+ width: 648px;
+ }
+ input.span10,
+ textarea.span10,
+ .uneditable-input.span10 {
+ width: 586px;
+ }
+ input.span9,
+ textarea.span9,
+ .uneditable-input.span9 {
+ width: 524px;
+ }
+ input.span8,
+ textarea.span8,
+ .uneditable-input.span8 {
+ width: 462px;
+ }
+ input.span7,
+ textarea.span7,
+ .uneditable-input.span7 {
+ width: 400px;
+ }
+ input.span6,
+ textarea.span6,
+ .uneditable-input.span6 {
+ width: 338px;
+ }
+ input.span5,
+ textarea.span5,
+ .uneditable-input.span5 {
+ width: 276px;
+ }
+ input.span4,
+ textarea.span4,
+ .uneditable-input.span4 {
+ width: 214px;
+ }
+ input.span3,
+ textarea.span3,
+ .uneditable-input.span3 {
+ width: 152px;
+ }
+ input.span2,
+ textarea.span2,
+ .uneditable-input.span2 {
+ width: 90px;
+ }
+ input.span1,
+ textarea.span1,
+ .uneditable-input.span1 {
+ width: 28px;
+ }
+}
+
+@media (max-width: 767px) {
+ body {
+ padding-right: 20px;
+ padding-left: 20px;
+ }
+ .navbar-fixed-top,
+ .navbar-fixed-bottom {
+ margin-right: -20px;
+ margin-left: -20px;
+ }
+ .container-fluid {
+ padding: 0;
+ }
+ .dl-horizontal dt {
+ float: none;
+ width: auto;
+ clear: none;
+ text-align: left;
+ }
+ .dl-horizontal dd {
+ margin-left: 0;
+ }
+ .container {
+ width: auto;
+ }
+ .row-fluid {
+ width: 100%;
+ }
+ .row,
+ .thumbnails {
+ margin-left: 0;
+ }
+ .thumbnails > li {
+ float: none;
+ margin-left: 0;
+ }
+ [class*="span"],
+ .row-fluid [class*="span"] {
+ display: block;
+ float: none;
+ width: auto;
+ margin-left: 0;
+ }
+ .span12,
+ .row-fluid .span12 {
+ width: 100%;
+ -webkit-box-sizing: border-box;
+ -moz-box-sizing: border-box;
+ box-sizing: border-box;
+ }
+ .input-large,
+ .input-xlarge,
+ .input-xxlarge,
+ input[class*="span"],
+ select[class*="span"],
+ textarea[class*="span"],
+ .uneditable-input {
+ display: block;
+ width: 100%;
+ min-height: 30px;
+ -webkit-box-sizing: border-box;
+ -moz-box-sizing: border-box;
+ box-sizing: border-box;
+ }
+ .input-prepend input,
+ .input-append input,
+ .input-prepend input[class*="span"],
+ .input-append input[class*="span"] {
+ display: inline-block;
+ width: auto;
+ }
+ .modal {
+ position: fixed;
+ top: 20px;
+ right: 20px;
+ left: 20px;
+ width: auto;
+ margin: 0;
+ }
+ .modal.fade.in {
+ top: auto;
+ }
+}
+
+@media (max-width: 480px) {
+ .nav-collapse {
+ -webkit-transform: translate3d(0, 0, 0);
+ }
+ .page-header h1 small {
+ display: block;
+ line-height: 20px;
+ }
+ input[type="checkbox"],
+ input[type="radio"] {
+ border: 1px solid #ccc;
+ }
+ .form-horizontal .control-group > label {
+ float: none;
+ width: auto;
+ padding-top: 0;
+ text-align: left;
+ }
+ .form-horizontal .controls {
+ margin-left: 0;
+ }
+ .form-horizontal .control-list {
+ padding-top: 0;
+ }
+ .form-horizontal .form-actions {
+ padding-right: 10px;
+ padding-left: 10px;
+ }
+ .modal {
+ top: 10px;
+ right: 10px;
+ left: 10px;
+ }
+ .modal-header .close {
+ padding: 10px;
+ margin: -10px;
+ }
+ .carousel-caption {
+ position: static;
+ }
+}
+
+@media (max-width: 979px) {
+ body {
+ padding-top: 0;
+ }
+ .navbar-fixed-top,
+ .navbar-fixed-bottom {
+ position: static;
+ }
+ .navbar-fixed-top {
+ margin-bottom: 20px;
+ }
+ .navbar-fixed-bottom {
+ margin-top: 20px;
+ }
+ .navbar-fixed-top .navbar-inner,
+ .navbar-fixed-bottom .navbar-inner {
+ padding: 5px;
+ }
+ .navbar .container {
+ width: auto;
+ padding: 0;
+ }
+ .navbar .brand {
+ padding-right: 10px;
+ padding-left: 10px;
+ margin: 0 0 0 -5px;
+ }
+ .nav-collapse {
+ clear: both;
+ }
+ .nav-collapse .nav {
+ float: none;
+ margin: 0 0 10px;
+ }
+ .nav-collapse .nav > li {
+ float: none;
+ }
+ .nav-collapse .nav > li > a {
+ margin-bottom: 2px;
+ }
+ .nav-collapse .nav > .divider-vertical {
+ display: none;
+ }
+ .nav-collapse .nav .nav-header {
+ color: #555555;
+ text-shadow: none;
+ }
+ .nav-collapse .nav > li > a,
+ .nav-collapse .dropdown-menu a {
+ padding: 9px 15px;
+ font-weight: bold;
+ color: #555555;
+ -webkit-border-radius: 3px;
+ -moz-border-radius: 3px;
+ border-radius: 3px;
+ }
+ .nav-collapse .btn {
+ padding: 4px 10px 4px;
+ font-weight: normal;
+ -webkit-border-radius: 4px;
+ -moz-border-radius: 4px;
+ border-radius: 4px;
+ }
+ .nav-collapse .dropdown-menu li + li a {
+ margin-bottom: 2px;
+ }
+ .nav-collapse .nav > li > a:hover,
+ .nav-collapse .dropdown-menu a:hover {
+ background-color: #f2f2f2;
+ }
+ .navbar-inverse .nav-collapse .nav > li > a:hover,
+ .navbar-inverse .nav-collapse .dropdown-menu a:hover {
+ background-color: #111111;
+ }
+ .nav-collapse.in .btn-group {
+ padding: 0;
+ margin-top: 5px;
+ }
+ .nav-collapse .dropdown-menu {
+ position: static;
+ top: auto;
+ left: auto;
+ display: block;
+ float: none;
+ max-width: none;
+ padding: 0;
+ margin: 0 15px;
+ background-color: transparent;
+ border: none;
+ -webkit-border-radius: 0;
+ -moz-border-radius: 0;
+ border-radius: 0;
+ -webkit-box-shadow: none;
+ -moz-box-shadow: none;
+ box-shadow: none;
+ }
+ .nav-collapse .dropdown-menu:before,
+ .nav-collapse .dropdown-menu:after {
+ display: none;
+ }
+ .nav-collapse .dropdown-menu .divider {
+ display: none;
+ }
+ .nav-collapse .navbar-form,
+ .nav-collapse .navbar-search {
+ float: none;
+ padding: 10px 15px;
+ margin: 10px 0;
+ border-top: 1px solid #f2f2f2;
+ border-bottom: 1px solid #f2f2f2;
+ -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);
+ -moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);
+ box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.1);
+ }
+ .navbar .nav-collapse .nav.pull-right {
+ float: none;
+ margin-left: 0;
+ }
+ .nav-collapse,
+ .nav-collapse.collapse {
+ height: 0;
+ overflow: hidden;
+ }
+ .navbar .btn-navbar {
+ display: block;
+ }
+ .navbar-static .navbar-inner {
+ padding-right: 10px;
+ padding-left: 10px;
+ }
+}
+
+@media (min-width: 980px) {
+ .nav-collapse.collapse {
+ height: auto !important;
+ overflow: visible !important;
+ }
+}
diff --git a/docs/css/bootstrap-responsive.min.css b/docs/css/bootstrap-responsive.min.css
new file mode 100644
index 0000000000..ab59da3427
--- /dev/null
+++ b/docs/css/bootstrap-responsive.min.css
@@ -0,0 +1,9 @@
+/*!
+ * Bootstrap Responsive v2.1.0
+ *
+ * Copyright 2012 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Designed and built with all the love in the world @twitter by @mdo and @fat.
+ */.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}.hidden{display:none;visibility:hidden}.visible-phone{display:none!important}.visible-tablet{display:none!important}.hidden-desktop{display:none!important}.visible-desktop{display:inherit!important}@media(min-width:768px) and (max-width:979px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-tablet{display:inherit!important}.hidden-tablet{display:none!important}}@media(max-width:767px){.hidden-desktop{display:inherit!important}.visible-desktop{display:none!important}.visible-phone{display:inherit!important}.hidden-phone{display:none!important}}@media(min-width:1200px){.row{margin-left:-30px;*zoom:1}.row:before,.row:after{display:table;line-height:0;content:""}.row:after{clear:both}[class*="span"]{float:left;margin-left:30px}.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:1170px}.span12{width:1170px}.span11{width:1070px}.span10{width:970px}.span9{width:870px}.span8{width:770px}.span7{width:670px}.span6{width:570px}.span5{width:470px}.span4{width:370px}.span3{width:270px}.span2{width:170px}.span1{width:70px}.offset12{margin-left:1230px}.offset11{margin-left:1130px}.offset10{margin-left:1030px}.offset9{margin-left:930px}.offset8{margin-left:830px}.offset7{margin-left:730px}.offset6{margin-left:630px}.offset5{margin-left:530px}.offset4{margin-left:430px}.offset3{margin-left:330px}.offset2{margin-left:230px}.offset1{margin-left:130px}.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.564102564102564%;*margin-left:2.5109110747408616%;-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.45299145299145%;*width:91.39979996362975%}.row-fluid .span10{width:82.90598290598291%;*width:82.8527914166212%}.row-fluid .span9{width:74.35897435897436%;*width:74.30578286961266%}.row-fluid .span8{width:65.81196581196582%;*width:65.75877432260411%}.row-fluid .span7{width:57.26495726495726%;*width:57.21176577559556%}.row-fluid .span6{width:48.717948717948715%;*width:48.664757228587014%}.row-fluid .span5{width:40.17094017094017%;*width:40.11774868157847%}.row-fluid .span4{width:31.623931623931625%;*width:31.570740134569924%}.row-fluid .span3{width:23.076923076923077%;*width:23.023731587561375%}.row-fluid .span2{width:14.52991452991453%;*width:14.476723040552828%}.row-fluid .span1{width:5.982905982905983%;*width:5.929714493544281%}.row-fluid .offset12{margin-left:105.12820512820512%;*margin-left:105.02182214948171%}.row-fluid .offset12:first-child{margin-left:102.56410256410257%;*margin-left:102.45771958537915%}.row-fluid .offset11{margin-left:96.58119658119658%;*margin-left:96.47481360247316%}.row-fluid .offset11:first-child{margin-left:94.01709401709402%;*margin-left:93.91071103837061%}.row-fluid .offset10{margin-left:88.03418803418803%;*margin-left:87.92780505546462%}.row-fluid .offset10:first-child{margin-left:85.47008547008548%;*margin-left:85.36370249136206%}.row-fluid .offset9{margin-left:79.48717948717949%;*margin-left:79.38079650845607%}.row-fluid .offset9:first-child{margin-left:76.92307692307693%;*margin-left:76.81669394435352%}.row-fluid .offset8{margin-left:70.94017094017094%;*margin-left:70.83378796144753%}.row-fluid .offset8:first-child{margin-left:68.37606837606839%;*margin-left:68.26968539734497%}.row-fluid .offset7{margin-left:62.393162393162385%;*margin-left:62.28677941443899%}.row-fluid .offset7:first-child{margin-left:59.82905982905982%;*margin-left:59.72267685033642%}.row-fluid .offset6{margin-left:53.84615384615384%;*margin-left:53.739770867430444%}.row-fluid .offset6:first-child{margin-left:51.28205128205128%;*margin-left:51.175668303327875%}.row-fluid .offset5{margin-left:45.299145299145295%;*margin-left:45.1927623204219%}.row-fluid .offset5:first-child{margin-left:42.73504273504273%;*margin-left:42.62865975631933%}.row-fluid .offset4{margin-left:36.75213675213675%;*margin-left:36.645753773413354%}.row-fluid .offset4:first-child{margin-left:34.18803418803419%;*margin-left:34.081651209310785%}.row-fluid .offset3{margin-left:28.205128205128204%;*margin-left:28.0987452264048%}.row-fluid .offset3:first-child{margin-left:25.641025641025642%;*margin-left:25.53464266230224%}.row-fluid .offset2{margin-left:19.65811965811966%;*margin-left:19.551736679396257%}.row-fluid .offset2:first-child{margin-left:17.094017094017094%;*margin-left:16.98763411529369%}.row-fluid .offset1{margin-left:11.11111111111111%;*margin-left:11.004728132387708%}.row-fluid .offset1:first-child{margin-left:8.547008547008547%;*margin-left:8.440625568285142%}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:30px}input.span12,textarea.span12,.uneditable-input.span12{width:1156px}input.span11,textarea.span11,.uneditable-input.span11{width:1056px}input.span10,textarea.span10,.uneditable-input.span10{width:956px}input.span9,textarea.span9,.uneditable-input.span9{width:856px}input.span8,textarea.span8,.uneditable-input.span8{width:756px}input.span7,textarea.span7,.uneditable-input.span7{width:656px}input.span6,textarea.span6,.uneditable-input.span6{width:556px}input.span5,textarea.span5,.uneditable-input.span5{width:456px}input.span4,textarea.span4,.uneditable-input.span4{width:356px}input.span3,textarea.span3,.uneditable-input.span3{width:256px}input.span2,textarea.span2,.uneditable-input.span2{width:156px}input.span1,textarea.span1,.uneditable-input.span1{width:56px}.thumbnails{margin-left:-30px}.thumbnails>li{margin-left:30px}.row-fluid .thumbnails{margin-left:0}}@media(min-width:768px) and (max-width:979px){.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:724px}.span12{width:724px}.span11{width:662px}.span10{width:600px}.span9{width:538px}.span8{width:476px}.span7{width:414px}.span6{width:352px}.span5{width:290px}.span4{width:228px}.span3{width:166px}.span2{width:104px}.span1{width:42px}.offset12{margin-left:764px}.offset11{margin-left:702px}.offset10{margin-left:640px}.offset9{margin-left:578px}.offset8{margin-left:516px}.offset7{margin-left:454px}.offset6{margin-left:392px}.offset5{margin-left:330px}.offset4{margin-left:268px}.offset3{margin-left:206px}.offset2{margin-left:144px}.offset1{margin-left:82px}.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.7624309392265194%;*margin-left:2.709239449864817%;-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.43646408839778%;*width:91.38327259903608%}.row-fluid .span10{width:82.87292817679558%;*width:82.81973668743387%}.row-fluid .span9{width:74.30939226519337%;*width:74.25620077583166%}.row-fluid .span8{width:65.74585635359117%;*width:65.69266486422946%}.row-fluid .span7{width:57.18232044198895%;*width:57.12912895262725%}.row-fluid .span6{width:48.61878453038674%;*width:48.56559304102504%}.row-fluid .span5{width:40.05524861878453%;*width:40.00205712942283%}.row-fluid .span4{width:31.491712707182323%;*width:31.43852121782062%}.row-fluid .span3{width:22.92817679558011%;*width:22.87498530621841%}.row-fluid .span2{width:14.3646408839779%;*width:14.311449394616199%}.row-fluid .span1{width:5.801104972375691%;*width:5.747913483013988%}.row-fluid .offset12{margin-left:105.52486187845304%;*margin-left:105.41847889972962%}.row-fluid .offset12:first-child{margin-left:102.76243093922652%;*margin-left:102.6560479605031%}.row-fluid .offset11{margin-left:96.96132596685082%;*margin-left:96.8549429881274%}.row-fluid .offset11:first-child{margin-left:94.1988950276243%;*margin-left:94.09251204890089%}.row-fluid .offset10{margin-left:88.39779005524862%;*margin-left:88.2914070765252%}.row-fluid .offset10:first-child{margin-left:85.6353591160221%;*margin-left:85.52897613729868%}.row-fluid .offset9{margin-left:79.8342541436464%;*margin-left:79.72787116492299%}.row-fluid .offset9:first-child{margin-left:77.07182320441989%;*margin-left:76.96544022569647%}.row-fluid .offset8{margin-left:71.2707182320442%;*margin-left:71.16433525332079%}.row-fluid .offset8:first-child{margin-left:68.50828729281768%;*margin-left:68.40190431409427%}.row-fluid .offset7{margin-left:62.70718232044199%;*margin-left:62.600799341718584%}.row-fluid .offset7:first-child{margin-left:59.94475138121547%;*margin-left:59.838368402492065%}.row-fluid .offset6{margin-left:54.14364640883978%;*margin-left:54.037263430116376%}.row-fluid .offset6:first-child{margin-left:51.38121546961326%;*margin-left:51.27483249088986%}.row-fluid .offset5{margin-left:45.58011049723757%;*margin-left:45.47372751851417%}.row-fluid .offset5:first-child{margin-left:42.81767955801105%;*margin-left:42.71129657928765%}.row-fluid .offset4{margin-left:37.01657458563536%;*margin-left:36.91019160691196%}.row-fluid .offset4:first-child{margin-left:34.25414364640884%;*margin-left:34.14776066768544%}.row-fluid .offset3{margin-left:28.45303867403315%;*margin-left:28.346655695309746%}.row-fluid .offset3:first-child{margin-left:25.69060773480663%;*margin-left:25.584224756083227%}.row-fluid .offset2{margin-left:19.88950276243094%;*margin-left:19.783119783707537%}.row-fluid .offset2:first-child{margin-left:17.12707182320442%;*margin-left:17.02068884448102%}.row-fluid .offset1{margin-left:11.32596685082873%;*margin-left:11.219583872105325%}.row-fluid .offset1:first-child{margin-left:8.56353591160221%;*margin-left:8.457152932878806%}input,textarea,.uneditable-input{margin-left:0}.controls-row [class*="span"]+[class*="span"]{margin-left:20px}input.span12,textarea.span12,.uneditable-input.span12{width:710px}input.span11,textarea.span11,.uneditable-input.span11{width:648px}input.span10,textarea.span10,.uneditable-input.span10{width:586px}input.span9,textarea.span9,.uneditable-input.span9{width:524px}input.span8,textarea.span8,.uneditable-input.span8{width:462px}input.span7,textarea.span7,.uneditable-input.span7{width:400px}input.span6,textarea.span6,.uneditable-input.span6{width:338px}input.span5,textarea.span5,.uneditable-input.span5{width:276px}input.span4,textarea.span4,.uneditable-input.span4{width:214px}input.span3,textarea.span3,.uneditable-input.span3{width:152px}input.span2,textarea.span2,.uneditable-input.span2{width:90px}input.span1,textarea.span1,.uneditable-input.span1{width:28px}}@media(max-width:767px){body{padding-right:20px;padding-left:20px}.navbar-fixed-top,.navbar-fixed-bottom{margin-right:-20px;margin-left:-20px}.container-fluid{padding:0}.dl-horizontal dt{float:none;width:auto;clear:none;text-align:left}.dl-horizontal dd{margin-left:0}.container{width:auto}.row-fluid{width:100%}.row,.thumbnails{margin-left:0}.thumbnails>li{float:none;margin-left:0}[class*="span"],.row-fluid [class*="span"]{display:block;float:none;width:auto;margin-left:0}.span12,.row-fluid .span12{width:100%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box}.input-prepend input,.input-append input,.input-prepend input[class*="span"],.input-append input[class*="span"]{display:inline-block;width:auto}.modal{position:fixed;top:20px;right:20px;left:20px;width:auto;margin:0}.modal.fade.in{top:auto}}@media(max-width:480px){.nav-collapse{-webkit-transform:translate3d(0,0,0)}.page-header h1 small{display:block;line-height:20px}input[type="checkbox"],input[type="radio"]{border:1px solid #ccc}.form-horizontal .control-group>label{float:none;width:auto;padding-top:0;text-align:left}.form-horizontal .controls{margin-left:0}.form-horizontal .control-list{padding-top:0}.form-horizontal .form-actions{padding-right:10px;padding-left:10px}.modal{top:10px;right:10px;left:10px}.modal-header .close{padding:10px;margin:-10px}.carousel-caption{position:static}}@media(max-width:979px){body{padding-top:0}.navbar-fixed-top,.navbar-fixed-bottom{position:static}.navbar-fixed-top{margin-bottom:20px}.navbar-fixed-bottom{margin-top:20px}.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px}.navbar .container{width:auto;padding:0}.navbar .brand{padding-right:10px;padding-left:10px;margin:0 0 0 -5px}.nav-collapse{clear:both}.nav-collapse .nav{float:none;margin:0 0 10px}.nav-collapse .nav>li{float:none}.nav-collapse .nav>li>a{margin-bottom:2px}.nav-collapse .nav>.divider-vertical{display:none}.nav-collapse .nav .nav-header{color:#555;text-shadow:none}.nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:9px 15px;font-weight:bold;color:#555;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px}.nav-collapse .btn{padding:4px 10px 4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px}.nav-collapse .dropdown-menu li+li a{margin-bottom:2px}.nav-collapse .nav>li>a:hover,.nav-collapse .dropdown-menu a:hover{background-color:#f2f2f2}.navbar-inverse .nav-collapse .nav>li>a:hover,.navbar-inverse .nav-collapse .dropdown-menu a:hover{background-color:#111}.nav-collapse.in .btn-group{padding:0;margin-top:5px}.nav-collapse .dropdown-menu{position:static;top:auto;left:auto;display:block;float:none;max-width:none;padding:0;margin:0 15px;background-color:transparent;border:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none}.nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none}.nav-collapse .dropdown-menu .divider{display:none}.nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:10px 15px;margin:10px 0;border-top:1px solid #f2f2f2;border-bottom:1px solid #f2f2f2;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1);box-shadow:inset 0 1px 0 rgba(255,255,255,0.1),0 1px 0 rgba(255,255,255,0.1)}.navbar .nav-collapse .nav.pull-right{float:none;margin-left:0}.nav-collapse,.nav-collapse.collapse{height:0;overflow:hidden}.navbar .btn-navbar{display:block}.navbar-static .navbar-inner{padding-right:10px;padding-left:10px}}@media(min-width:980px){.nav-collapse.collapse{height:auto!important;overflow:visible!important}}
diff --git a/docs/css/bootstrap.css b/docs/css/bootstrap.css
new file mode 100644
index 0000000000..0664207aee
--- /dev/null
+++ b/docs/css/bootstrap.css
@@ -0,0 +1,5624 @@
+/*!
+ * Bootstrap v2.1.0
+ *
+ * Copyright 2012 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * 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: #333333;
+ background-color: #ffffff;
+}
+
+a {
+ color: #0088cc;
+ 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: #999999;
+}
+
+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: #999999;
+}
+
+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 #eeeeee;
+}
+
+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 #eeeeee;
+ border-bottom: 1px solid #ffffff;
+}
+
+abbr[title] {
+ cursor: help;
+ border-bottom: 1px dotted #999999;
+}
+
+abbr.initialism {
+ font-size: 90%;
+ text-transform: uppercase;
+}
+
+blockquote {
+ padding: 0 0 0 15px;
+ margin: 0 0 20px;
+ border-left: 5px solid #eeeeee;
+}
+
+blockquote p {
+ margin-bottom: 0;
+ font-size: 16px;
+ font-weight: 300;
+ line-height: 25px;
+}
+
+blockquote small {
+ display: block;
+ line-height: 20px;
+ color: #999999;
+}
+
+blockquote small:before {
+ content: '\2014 \00A0';
+}
+
+blockquote.pull-right {
+ float: right;
+ padding-right: 15px;
+ padding-left: 0;
+ border-right: 5px solid #eeeeee;
+ 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: #333333;
+ -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: #333333;
+ border: 0;
+ border-bottom: 1px solid #e5e5e5;
+}
+
+legend small {
+ font-size: 15px;
+ color: #999999;
+}
+
+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: #555555;
+ -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: #ffffff;
+ border: 1px solid #cccccc;
+ -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 0.2s, box-shadow linear 0.2s;
+ -moz-transition: border linear 0.2s, box-shadow linear 0.2s;
+ -o-transition: border linear 0.2s, box-shadow linear 0.2s;
+ transition: border linear 0.2s, box-shadow linear 0.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;
+ /* IE6-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;
+ /* In IE7, the height of the select element cannot be changed by height, only font-size */
+
+ *margin-top: 4px;
+ /* For IE7, add top margin to align select with labels */
+
+ line-height: 30px;
+}
+
+select {
+ width: 220px;
+ background-color: #ffffff;
+ 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: #999999;
+ cursor: not-allowed;
+ background-color: #fcfcfc;
+ border-color: #cccccc;
+ -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: #999999;
+}
+
+input:-ms-input-placeholder,
+textarea:-ms-input-placeholder {
+ color: #999999;
+}
+
+input::-webkit-input-placeholder,
+textarea::-webkit-input-placeholder {
+ color: #999999;
+}
+
+.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: #eeeeee;
+}
+
+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 #ffffff;
+ background-color: #eeeeee;
+ 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;
+ /* IE7-8 doesn't have border-radius, so don't indent the padding */
+
+ margin-bottom: 0;
+ -webkit-border-radius: 15px;
+ -moz-border-radius: 15px;
+ border-radius: 15px;
+}
+
+/* Allow for input prepend/append in search forms */
+
+.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 #dddddd;
+}
+
+.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 #dddddd;
+}
+
+.table-condensed th,
+.table-condensed td {
+ padding: 4px 5px;
+}
+
+.table-bordered {
+ border: 1px solid #dddddd;
+ 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 #dddddd;
+}
+
+.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;
+}
+
+/* White icons with optional class, or on hover/active states of certain elements */
+
+.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 #000000;
+ 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: #ffffff;
+ 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 #ffffff;
+}
+
+.dropdown-menu a {
+ display: block;
+ padding: 3px 20px;
+ clear: both;
+ font-weight: normal;
+ line-height: 20px;
+ color: #333333;
+ white-space: nowrap;
+}
+
+.dropdown-menu li > a:hover,
+.dropdown-menu li > a:focus,
+.dropdown-submenu:hover > a {
+ color: #ffffff;
+ text-decoration: none;
+ background-color: #0088cc;
+ background-color: #0081c2;
+ background-image: -moz-linear-gradient(top, #0088cc, #0077b3);
+ background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3));
+ background-image: -webkit-linear-gradient(top, #0088cc, #0077b3);
+ background-image: -o-linear-gradient(top, #0088cc, #0077b3);
+ background-image: linear-gradient(to bottom, #0088cc, #0077b3);
+ background-repeat: repeat-x;
+ filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0);
+}
+
+.dropdown-menu .active > a,
+.dropdown-menu .active > a:hover {
+ color: #ffffff;
+ text-decoration: none;
+ background-color: #0088cc;
+ background-color: #0081c2;
+ background-image: linear-gradient(to bottom, #0088cc, #0077b3);
+ background-image: -moz-linear-gradient(top, #0088cc, #0077b3);
+ background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3));
+ background-image: -webkit-linear-gradient(top, #0088cc, #0077b3);
+ background-image: -o-linear-gradient(top, #0088cc, #0077b3);
+ background-repeat: repeat-x;
+ outline: 0;
+ filter: progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0);
+}
+
+.dropdown-menu .disabled > a,
+.dropdown-menu .disabled > a:hover {
+ color: #999999;
+}
+
+.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 #000000;
+ 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: #cccccc;
+ border-style: solid;
+ border-width: 5px 0 5px 5px;
+ content: " ";
+}
+
+.dropdown-submenu:hover > a:after {
+ border-left-color: #ffffff;
+}
+
+.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 0.15s linear;
+ -moz-transition: opacity 0.15s linear;
+ -o-transition: opacity 0.15s linear;
+ transition: opacity 0.15s linear;
+}
+
+.fade.in {
+ opacity: 1;
+}
+
+.collapse {
+ position: relative;
+ height: 0;
+ overflow: hidden;
+ overflow: visible \9;
+ -webkit-transition: height 0.35s ease;
+ -moz-transition: height 0.35s ease;
+ -o-transition: height 0.35s ease;
+ transition: height 0.35s ease;
+}
+
+.collapse.in {
+ height: auto;
+}
+
+.close {
+ float: right;
+ font-size: 20px;
+ font-weight: bold;
+ line-height: 20px;
+ color: #000000;
+ text-shadow: 0 1px 0 #ffffff;
+ opacity: 0.2;
+ filter: alpha(opacity=20);
+}
+
+.close:hover {
+ color: #000000;
+ text-decoration: none;
+ cursor: pointer;
+ opacity: 0.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: #333333;
+ 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(#ffffff), to(#e6e6e6));
+ background-image: -webkit-linear-gradient(top, #ffffff, #e6e6e6);
+ background-image: -o-linear-gradient(top, #ffffff, #e6e6e6);
+ background-image: linear-gradient(to bottom, #ffffff, #e6e6e6);
+ background-image: -moz-linear-gradient(top, #ffffff, #e6e6e6);
+ background-repeat: repeat-x;
+ border: 1px solid #bbbbbb;
+ *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: #333333;
+ background-color: #e6e6e6;
+ *background-color: #d9d9d9;
+}
+
+.btn:active,
+.btn.active {
+ background-color: #cccccc \9;
+}
+
+.btn:first-child {
+ *margin-left: 0;
+}
+
+.btn:hover {
+ color: #333333;
+ text-decoration: none;
+ background-color: #e6e6e6;
+ *background-color: #d9d9d9;
+ /* Buttons in IE7 don't get borders, so darken on hover */
+
+ background-position: 0 -15px;
+ -webkit-transition: background-position 0.1s linear;
+ -moz-transition: background-position 0.1s linear;
+ -o-transition: background-position 0.1s linear;
+ transition: background-position 0.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: 0.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: #ffffff;
+ text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+ background-color: #006dcc;
+ *background-color: #0044cc;
+ background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0044cc));
+ background-image: -webkit-linear-gradient(top, #0088cc, #0044cc);
+ background-image: -o-linear-gradient(top, #0088cc, #0044cc);
+ background-image: linear-gradient(to bottom, #0088cc, #0044cc);
+ background-image: -moz-linear-gradient(top, #0088cc, #0044cc);
+ background-repeat: repeat-x;
+ border-color: #0044cc #0044cc #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: #ffffff;
+ background-color: #0044cc;
+ *background-color: #003bb3;
+}
+
+.btn-primary:active,
+.btn-primary.active {
+ background-color: #003399 \9;
+}
+
+.btn-warning {
+ color: #ffffff;
+ 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: #ffffff;
+ background-color: #f89406;
+ *background-color: #df8505;
+}
+
+.btn-warning:active,
+.btn-warning.active {
+ background-color: #c67605 \9;
+}
+
+.btn-danger {
+ color: #ffffff;
+ 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: #ffffff;
+ background-color: #bd362f;
+ *background-color: #a9302a;
+}
+
+.btn-danger:active,
+.btn-danger.active {
+ background-color: #942a25 \9;
+}
+
+.btn-success {
+ color: #ffffff;
+ 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: #ffffff;
+ background-color: #51a351;
+ *background-color: #499249;
+}
+
+.btn-success:active,
+.btn-success.active {
+ background-color: #408140 \9;
+}
+
+.btn-info {
+ color: #ffffff;
+ 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: #ffffff;
+ background-color: #2f96b4;
+ *background-color: #2a85a0;
+}
+
+.btn-info:active,
+.btn-info.active {
+ background-color: #24748c \9;
+}
+
+.btn-inverse {
+ color: #ffffff;
+ text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+ background-color: #363636;
+ *background-color: #222222;
+ background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#444444), to(#222222));
+ background-image: -webkit-linear-gradient(top, #444444, #222222);
+ background-image: -o-linear-gradient(top, #444444, #222222);
+ background-image: linear-gradient(to bottom, #444444, #222222);
+ background-image: -moz-linear-gradient(top, #444444, #222222);
+ background-repeat: repeat-x;
+ border-color: #222222 #222222 #000000;
+ 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: #ffffff;
+ background-color: #222222;
+ *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: #0088cc;
+ 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;
+ /* IE7 inline-block hack */
+
+ *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: #0044cc;
+}
+
+.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: #222222;
+}
+
+.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 #000000;
+}
+
+.btn-primary .caret,
+.btn-warning .caret,
+.btn-danger .caret,
+.btn-info .caret,
+.btn-success .caret,
+.btn-inverse .caret {
+ border-top-color: #ffffff;
+ border-bottom-color: #ffffff;
+}
+
+.btn-group-vertical {
+ display: inline-block;
+ *display: inline;
+ /* IE7 inline-block hack */
+
+ *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: #eeeeee;
+}
+
+.nav > .pull-right {
+ float: right;
+}
+
+.nav-header {
+ display: block;
+ padding: 3px 15px;
+ font-size: 11px;
+ font-weight: bold;
+ line-height: 20px;
+ color: #999999;
+ 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: #ffffff;
+ text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.2);
+ background-color: #0088cc;
+}
+
+.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 #ffffff;
+}
+
+.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: #eeeeee #eeeeee #dddddd;
+}
+
+.nav-tabs > .active > a,
+.nav-tabs > .active > a:hover {
+ color: #555555;
+ cursor: default;
+ background-color: #ffffff;
+ 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: #ffffff;
+ background-color: #0088cc;
+}
+
+.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: #0088cc;
+ border-bottom-color: #0088cc;
+}
+
+.nav .dropdown-toggle:hover .caret {
+ border-top-color: #005580;
+ border-bottom-color: #005580;
+}
+
+/* move down carets for tabs */
+
+.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: #555555;
+ border-bottom-color: #555555;
+}
+
+.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: #ffffff;
+ background-color: #999999;
+ border-color: #999999;
+}
+
+.nav li.dropdown.open .caret,
+.nav li.dropdown.open.active .caret,
+.nav li.dropdown.open a:hover .caret {
+ border-top-color: #ffffff;
+ border-bottom-color: #ffffff;
+ opacity: 1;
+ filter: alpha(opacity=100);
+}
+
+.tabs-stacked .open > a:hover {
+ border-color: #999999;
+}
+
+.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: #eeeeee #dddddd #eeeeee #eeeeee;
+}
+
+.tabs-left > .nav-tabs .active > a,
+.tabs-left > .nav-tabs .active > a:hover {
+ border-color: #ddd transparent #ddd #ddd;
+ *border-right-color: #ffffff;
+}
+
+.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: #eeeeee #eeeeee #eeeeee #dddddd;
+}
+
+.tabs-right > .nav-tabs .active > a,
+.tabs-right > .nav-tabs .active > a:hover {
+ border-color: #ddd #ddd #ddd transparent;
+ *border-left-color: #ffffff;
+}
+
+.nav > .disabled > a {
+ color: #999999;
+}
+
+.nav > .disabled > a:hover {
+ text-decoration: none;
+ cursor: default;
+ background-color: transparent;
+}
+
+.navbar {
+ *position: relative;
+ *z-index: 2;
+ margin-bottom: 20px;
+ overflow: visible;
+ color: #555555;
+}
+
+.navbar-inner {
+ min-height: 40px;
+ padding-right: 20px;
+ padding-left: 20px;
+ background-color: #fafafa;
+ background-image: -moz-linear-gradient(top, #ffffff, #f2f2f2);
+ background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#f2f2f2));
+ background-image: -webkit-linear-gradient(top, #ffffff, #f2f2f2);
+ background-image: -o-linear-gradient(top, #ffffff, #f2f2f2);
+ background-image: linear-gradient(to bottom, #ffffff, #f2f2f2);
+ 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='#fff2f2f2', 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: #555555;
+ text-shadow: 0 1px 0 #ffffff;
+}
+
+.navbar .brand:hover {
+ text-decoration: none;
+}
+
+.navbar-text {
+ margin-bottom: 0;
+ line-height: 40px;
+}
+
+.navbar-link {
+ color: #555555;
+}
+
+.navbar-link:hover {
+ color: #333333;
+}
+
+.navbar .divider-vertical {
+ height: 40px;
+ margin: 0 9px;
+ border-right: 1px solid #ffffff;
+ 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: #555555;
+ text-decoration: none;
+ text-shadow: 0 1px 0 #ffffff;
+}
+
+.navbar .nav .dropdown-toggle .caret {
+ margin-top: 8px;
+}
+
+.navbar .nav > li > a:focus,
+.navbar .nav > li > a:hover {
+ color: #333333;
+ text-decoration: none;
+ background-color: transparent;
+}
+
+.navbar .nav > .active > a,
+.navbar .nav > .active > a:hover,
+.navbar .nav > .active > a:focus {
+ color: #555555;
+ 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: #ffffff;
+ 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: #ffffff;
+ background-color: #e5e5e5;
+ *background-color: #d9d9d9;
+}
+
+.navbar .btn-navbar:active,
+.navbar .btn-navbar.active {
+ background-color: #cccccc \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 #ffffff;
+ 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 #ffffff;
+ 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: #555555;
+ background-color: #e5e5e5;
+}
+
+.navbar .nav li.dropdown > .dropdown-toggle .caret {
+ border-top-color: #555555;
+ border-bottom-color: #555555;
+}
+
+.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: #555555;
+ border-bottom-color: #555555;
+}
+
+.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: #999999;
+}
+
+.navbar-inverse .navbar-inner {
+ background-color: #1b1b1b;
+ background-image: -moz-linear-gradient(top, #222222, #111111);
+ background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#222222), to(#111111));
+ background-image: -webkit-linear-gradient(top, #222222, #111111);
+ background-image: -o-linear-gradient(top, #222222, #111111);
+ background-image: linear-gradient(to bottom, #222222, #111111);
+ 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: #999999;
+ text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+}
+
+.navbar-inverse .brand:hover,
+.navbar-inverse .nav > li > a:hover {
+ color: #ffffff;
+}
+
+.navbar-inverse .nav > li > a:focus,
+.navbar-inverse .nav > li > a:hover {
+ color: #ffffff;
+ background-color: transparent;
+}
+
+.navbar-inverse .nav .active > a,
+.navbar-inverse .nav .active > a:hover,
+.navbar-inverse .nav .active > a:focus {
+ color: #ffffff;
+ background-color: #111111;
+}
+
+.navbar-inverse .navbar-link {
+ color: #999999;
+}
+
+.navbar-inverse .navbar-link:hover {
+ color: #ffffff;
+}
+
+.navbar-inverse .divider-vertical {
+ border-right-color: #222222;
+ border-left-color: #111111;
+}
+
+.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: #ffffff;
+ background-color: #111111;
+}
+
+.navbar-inverse .nav li.dropdown > .dropdown-toggle .caret {
+ border-top-color: #999999;
+ border-bottom-color: #999999;
+}
+
+.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: #ffffff;
+ border-bottom-color: #ffffff;
+}
+
+.navbar-inverse .navbar-search .search-query {
+ color: #ffffff;
+ background-color: #515151;
+ border-color: #111111;
+ -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: #cccccc;
+}
+
+.navbar-inverse .navbar-search .search-query:-ms-input-placeholder {
+ color: #cccccc;
+}
+
+.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder {
+ color: #cccccc;
+}
+
+.navbar-inverse .navbar-search .search-query:focus,
+.navbar-inverse .navbar-search .search-query.focused {
+ padding: 5px 15px;
+ color: #333333;
+ text-shadow: 0 1px 0 #ffffff;
+ background-color: #ffffff;
+ 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: #ffffff;
+ 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 #000000;
+ 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: #ffffff;
+ background-color: #040404;
+ *background-color: #000000;
+}
+
+.navbar-inverse .btn-navbar:active,
+.navbar-inverse .btn-navbar.active {
+ background-color: #000000 \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 #ffffff;
+ *zoom: 1;
+}
+
+.breadcrumb .divider {
+ padding: 0 5px;
+ color: #ccc;
+}
+
+.breadcrumb .active {
+ color: #999999;
+}
+
+.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: #ffffff;
+ border: 1px solid #dddddd;
+ border-left-width: 0;
+}
+
+.pagination a:hover,
+.pagination .active a,
+.pagination .active span {
+ background-color: #f5f5f5;
+}
+
+.pagination .active a,
+.pagination .active span {
+ color: #999999;
+ cursor: default;
+}
+
+.pagination .disabled span,
+.pagination .disabled a,
+.pagination .disabled a:hover {
+ color: #999999;
+ 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: #999999;
+ 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: #000000;
+}
+
+.modal-backdrop.fade {
+ opacity: 0;
+}
+
+.modal-backdrop,
+.modal-backdrop.fade.in {
+ opacity: 0.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: #ffffff;
+ 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 0.3s linear, top 0.3s ease-out;
+ -moz-transition: opacity 0.3s linear, top 0.3s ease-out;
+ -o-transition: opacity 0.3s linear, top 0.3s ease-out;
+ transition: opacity 0.3s linear, top 0.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 #ffffff;
+ -moz-box-shadow: inset 0 1px 0 #ffffff;
+ box-shadow: inset 0 1px 0 #ffffff;
+}
+
+.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: 0.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: #ffffff;
+ text-align: center;
+ text-decoration: none;
+ background-color: #000000;
+ -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: #000000;
+ border-width: 5px 5px 0;
+}
+
+.tooltip.right .tooltip-arrow {
+ top: 50%;
+ left: 0;
+ margin-top: -5px;
+ border-right-color: #000000;
+ border-width: 5px 5px 5px 0;
+}
+
+.tooltip.left .tooltip-arrow {
+ top: 50%;
+ right: 0;
+ margin-top: -5px;
+ border-left-color: #000000;
+ border-width: 5px 0 5px 5px;
+}
+
+.tooltip.bottom .tooltip-arrow {
+ top: 0;
+ left: 50%;
+ margin-left: -5px;
+ border-bottom-color: #000000;
+ border-width: 0 5px 5px;
+}
+
+.popover {
+ position: absolute;
+ top: 0;
+ left: 0;
+ z-index: 1010;
+ display: none;
+ width: 236px;
+ padding: 1px;
+ background-color: #ffffff;
+ 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: #ffffff;
+ 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: #ffffff;
+ 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: #ffffff;
+ 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: #ffffff;
+ 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 0.2s ease-in-out;
+ -moz-transition: all 0.2s ease-in-out;
+ -o-transition: all 0.2s ease-in-out;
+ transition: all 0.2s ease-in-out;
+}
+
+a.thumbnail:hover {
+ border-color: #0088cc;
+ -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: #555555;
+}
+
+.label,
+.badge {
+ font-size: 11.844px;
+ font-weight: bold;
+ line-height: 14px;
+ color: #ffffff;
+ text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25);
+ white-space: nowrap;
+ vertical-align: baseline;
+ background-color: #999999;
+}
+
+.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: #ffffff;
+ 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: #333333;
+}
+
+.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: #ffffff;
+ 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 0.6s ease;
+ -moz-transition: width 0.6s ease;
+ -o-transition: width 0.6s ease;
+ transition: width 0.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: 0.6s ease-in-out left;
+ -moz-transition: 0.6s ease-in-out left;
+ -o-transition: 0.6s ease-in-out left;
+ transition: 0.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: #ffffff;
+ text-align: center;
+ background: #222222;
+ border: 3px solid #ffffff;
+ -webkit-border-radius: 23px;
+ -moz-border-radius: 23px;
+ border-radius: 23px;
+ opacity: 0.5;
+ filter: alpha(opacity=50);
+}
+
+.carousel-control.right {
+ right: 15px;
+ left: auto;
+}
+
+.carousel-control:hover {
+ color: #ffffff;
+ text-decoration: none;
+ opacity: 0.9;
+ filter: alpha(opacity=90);
+}
+
+.carousel-caption {
+ position: absolute;
+ right: 0;
+ bottom: 0;
+ left: 0;
+ padding: 15px;
+ background: #333333;
+ background: rgba(0, 0, 0, 0.75);
+}
+
+.carousel-caption h4,
+.carousel-caption p {
+ line-height: 20px;
+ color: #ffffff;
+}
+
+.carousel-caption h4 {
+ margin: 0 0 5px;
+}
+
+.carousel-caption p {
+ margin-bottom: 0;
+}
+
+.hero-unit {
+ padding: 60px;
+ margin-bottom: 30px;
+ background-color: #eeeeee;
+ -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/css/bootstrap.min.css b/docs/css/bootstrap.min.css
new file mode 100644
index 0000000000..3119038a6b
--- /dev/null
+++ b/docs/css/bootstrap.min.css
@@ -0,0 +1,9 @@
+/*!
+ * Bootstrap v2.1.0
+ *
+ * Copyright 2012 Twitter, Inc
+ * Licensed under the Apache License v2.0
+ * 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:#08c;background-color:#0081c2;background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-image:linear-gradient(to bottom,#08c,#0077b3);background-repeat:repeat-x;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',GradientType=0)}.dropdown-menu .active>a,.dropdown-menu .active>a:hover{color:#fff;text-decoration:none;background-color:#08c;background-color:#0081c2;background-image:linear-gradient(to bottom,#08c,#0077b3);background-image:-moz-linear-gradient(top,#08c,#0077b3);background-image:-webkit-gradient(linear,0 0,0 100%,from(#08c),to(#0077b3));background-image:-webkit-linear-gradient(top,#08c,#0077b3);background-image:-o-linear-gradient(top,#08c,#0077b3);background-repeat:repeat-x;outline:0;filter:progid:dximagetransform.microsoft.gradient(startColorstr='#ff0088cc',endColorstr='#ff0077b3',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,#f2f2f2);background-image:-webkit-gradient(linear,0 0,0 100%,from(#fff),to(#f2f2f2));background-image:-webkit-linear-gradient(top,#fff,#f2f2f2);background-image:-o-linear-gradient(top,#fff,#f2f2f2);background-image:linear-gradient(to bottom,#fff,#f2f2f2);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='#fff2f2f2',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/css/main.css b/docs/css/main.css
new file mode 100755
index 0000000000..31122d5633
--- /dev/null
+++ b/docs/css/main.css
@@ -0,0 +1,106 @@
+/* ==========================================================================
+ Author's custom styles
+ ========================================================================== */
+
+.navbar .brand {
+ height: 50px;
+ width: 110px;
+ margin-left: 1px;
+ padding: 0;
+}
+
+.version {
+ line-height: 30px;
+ vertical-align: bottom;
+ font-size: 12px;
+ padding: 0;
+ margin: 0;
+ font-weight: bold;
+ color: #777;
+}
+
+.navbar-inner {
+ padding-top: 2px;
+ height: 50px;
+}
+
+.navbar-inner .nav {
+ margin-top: 5px;
+ font-size: 15px;
+}
+
+.navbar .divider-vertical {
+ border-right-color: lightgray;
+}
+
+.navbar-text .version-text {
+ color: #555555;
+ padding: 5px;
+ margin-left: 10px;
+}
+
+body #content {
+ line-height: 1.6; /* Inspired by Github's wiki style */
+}
+
+.title {
+ font-size: 32px;
+}
+
+h1 {
+ font-size: 28px;
+ margin-top: 12px;
+}
+
+h2 {
+ font-size: 24px;
+ margin-top: 12px;
+}
+
+h3 {
+ font-size: 21px;
+ margin-top: 10px;
+}
+
+pre {
+ font-family: "Menlo", "Lucida Console", monospace;
+}
+
+code {
+ font-family: "Menlo", "Lucida Console", monospace;
+ background: white;
+ border: none;
+ padding: 0;
+ color: #444444;
+}
+
+a code {
+ color: #0088cc;
+}
+
+a:hover code {
+ color: #005580;
+ text-decoration: underline;
+}
+
+.container {
+ max-width: 914px;
+}
+
+/**
+ * Make dropdown menus in nav bars show on hover instead of click
+ * using solution at http://stackoverflow.com/questions/8878033/how-
+ * to-make-twitter-bootstrap-menu-dropdown-on-hover-rather-than-click
+ **/
+.dropdown-menu {
+ /* Remove the default 2px top margin which causes a small
+ gap between the hover trigger area and the popup menu */
+ margin-top: 0;
+}
+ul.nav li.dropdown:hover ul.dropdown-menu{
+ display: block;
+}
+a.menu:after, .dropdown-toggle:after {
+ content: none;
+}
+
diff --git a/docs/css/pygments-default.css b/docs/css/pygments-default.css
new file mode 100644
index 0000000000..6247cd8396
--- /dev/null
+++ b/docs/css/pygments-default.css
@@ -0,0 +1,76 @@
+/*
+Documentation for pygments (and Jekyll for that matter) is super sparse.
+To generate this, I had to run
+ `pygmentize -S default -f html > pygments-default.css`
+But first I had to install pygments via easy_install pygments
+
+I had to override the conflicting bootstrap style rules by linking to
+this stylesheet lower in the html than the bootstap css.
+
+Also, I was thrown off for a while at first when I was using markdown
+code block inside my {% highlight scala %} ... {% endhighlight %} tags
+(I was using 4 spaces for this), when it turns out that pygments will
+insert the code (or pre?) tags for you.
+*/
+
+.hll { background-color: #ffffcc }
+.c { color: #60a0b0; font-style: italic } /* Comment */
+.err { } /* Error */
+.k { color: #007020; font-weight: bold } /* Keyword */
+.o { color: #666666 } /* Operator */
+.cm { color: #60a0b0; font-style: italic } /* Comment.Multiline */
+.cp { color: #007020 } /* Comment.Preproc */
+.c1 { color: #60a0b0; font-style: italic } /* Comment.Single */
+.cs { color: #60a0b0; background-color: #fff0f0 } /* Comment.Special */
+.gd { color: #A00000 } /* Generic.Deleted */
+.ge { font-style: italic } /* Generic.Emph */
+.gr { color: #FF0000 } /* Generic.Error */
+.gh { color: #000080; font-weight: bold } /* Generic.Heading */
+.gi { color: #00A000 } /* Generic.Inserted */
+.go { color: #808080 } /* Generic.Output */
+.gp { color: #c65d09; font-weight: bold } /* Generic.Prompt */
+.gs { font-weight: bold } /* Generic.Strong */
+.gu { color: #800080; font-weight: bold } /* Generic.Subheading */
+.gt { color: #0040D0 } /* Generic.Traceback */
+.kc { color: #007020; font-weight: bold } /* Keyword.Constant */
+.kd { color: #007020; font-weight: bold } /* Keyword.Declaration */
+.kn { color: #007020; font-weight: bold } /* Keyword.Namespace */
+.kp { color: #007020 } /* Keyword.Pseudo */
+.kr { color: #007020; font-weight: bold } /* Keyword.Reserved */
+.kt { color: #902000 } /* Keyword.Type */
+.m { color: #40a070 } /* Literal.Number */
+.s { color: #4070a0 } /* Literal.String */
+.na { color: #4070a0 } /* Name.Attribute */
+.nb { color: #007020 } /* Name.Builtin */
+.nc { color: #0e84b5; font-weight: bold } /* Name.Class */
+.no { color: #60add5 } /* Name.Constant */
+.nd { color: #555555; font-weight: bold } /* Name.Decorator */
+.ni { color: #d55537; font-weight: bold } /* Name.Entity */
+.ne { color: #007020 } /* Name.Exception */
+.nf { color: #06287e } /* Name.Function */
+.nl { color: #002070; font-weight: bold } /* Name.Label */
+.nn { color: #0e84b5; font-weight: bold } /* Name.Namespace */
+.nt { color: #062873; font-weight: bold } /* Name.Tag */
+.nv { color: #bb60d5 } /* Name.Variable */
+.ow { color: #007020; font-weight: bold } /* Operator.Word */
+.w { color: #bbbbbb } /* Text.Whitespace */
+.mf { color: #40a070 } /* Literal.Number.Float */
+.mh { color: #40a070 } /* Literal.Number.Hex */
+.mi { color: #40a070 } /* Literal.Number.Integer */
+.mo { color: #40a070 } /* Literal.Number.Oct */
+.sb { color: #4070a0 } /* Literal.String.Backtick */
+.sc { color: #4070a0 } /* Literal.String.Char */
+.sd { color: #4070a0; font-style: italic } /* Literal.String.Doc */
+.s2 { color: #4070a0 } /* Literal.String.Double */
+.se { color: #4070a0; font-weight: bold } /* Literal.String.Escape */
+.sh { color: #4070a0 } /* Literal.String.Heredoc */
+.si { color: #70a0d0; font-style: italic } /* Literal.String.Interpol */
+.sx { color: #c65d09 } /* Literal.String.Other */
+.sr { color: #235388 } /* Literal.String.Regex */
+.s1 { color: #4070a0 } /* Literal.String.Single */
+.ss { color: #517918 } /* Literal.String.Symbol */
+.bp { color: #007020 } /* Name.Builtin.Pseudo */
+.vc { color: #bb60d5 } /* Name.Variable.Class */
+.vg { color: #bb60d5 } /* Name.Variable.Global */
+.vi { color: #bb60d5 } /* Name.Variable.Instance */
+.il { color: #40a070 } /* Literal.Number.Integer.Long */ \ No newline at end of file
diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md
new file mode 100644
index 0000000000..6e1f7fd3b1
--- /dev/null
+++ b/docs/ec2-scripts.md
@@ -0,0 +1,159 @@
+---
+layout: global
+title: Running Spark on EC2
+---
+
+The `spark-ec2` script, located in Spark's `ec2` directory, allows you
+to launch, manage and shut down Spark clusters on Amazon EC2. It automatically sets up Mesos, Spark and HDFS
+on the cluster for you.
+This guide describes how to use `spark-ec2` to launch clusters, how to run jobs on them, and how to shut them down.
+It assumes you've already signed up for an EC2 account on the [Amazon Web Services site](http://aws.amazon.com/).
+
+`spark-ec2` is designed to manage multiple named clusters. You can
+launch a new cluster (telling the script its size and giving it a name),
+shutdown an existing cluster, or log into a cluster. Each cluster is
+identified by placing its machines into EC2 security groups whose names
+are derived from the name of the cluster. For example, a cluster named
+`test` will contain a master node in a security group called
+`test-master`, and a number of slave nodes in a security group called
+`test-slaves`. The `spark-ec2` script will create these security groups
+for you based on the cluster name you request. You can also use them to
+identify machines belonging to each cluster in the Amazon EC2 Console.
+
+
+# Before You Start
+
+- Create an Amazon EC2 key pair for yourself. This can be done by
+ logging into your Amazon Web Services account through the [AWS
+ console](http://aws.amazon.com/console/), clicking Key Pairs on the
+ left sidebar, and creating and downloading a key. Make sure that you
+ set the permissions for the private key file to `600` (i.e. only you
+ can read and write it) so that `ssh` will work.
+- Whenever you want to use the `spark-ec2` script, set the environment
+ variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` to your
+ Amazon EC2 access key ID and secret access key. These can be
+ obtained from the [AWS homepage](http://aws.amazon.com/) by clicking
+ Account \> Security Credentials \> Access Credentials.
+
+# Launching a Cluster
+
+- Go into the `ec2` directory in the release of Spark you downloaded.
+- Run
+ `./spark-ec2 -k <keypair> -i <key-file> -s <num-slaves> launch <cluster-name>`,
+ where `<keypair>` is the name of your EC2 key pair (that you gave it
+ when you created it), `<key-file>` is the private key file for your
+ key pair, `<num-slaves>` is the number of slave nodes to launch (try
+ 1 at first), and `<cluster-name>` is the name to give to your
+ cluster.
+- After everything launches, check that Mesos is up and sees all the
+ slaves by going to the Mesos Web UI link printed at the end of the
+ script (`http://<master-hostname>:8080`).
+
+You can also run `./spark-ec2 --help` to see more usage options. The
+following options are worth pointing out:
+
+- `--instance-type=<INSTANCE_TYPE>` can be used to specify an EC2
+instance type to use. For now, the script only supports 64-bit instance
+types, and the default type is `m1.large` (which has 2 cores and 7.5 GB
+RAM). Refer to the Amazon pages about [EC2 instance
+types](http://aws.amazon.com/ec2/instance-types) and [EC2
+pricing](http://aws.amazon.com/ec2/#pricing) for information about other
+instance types.
+- `--zone=<EC2_ZONE>` can be used to specify an EC2 availability zone
+to launch instances in. Sometimes, you will get an error because there
+is not enough capacity in one zone, and you should try to launch in
+another. This happens mostly with the `m1.large` instance types;
+extra-large (both `m1.xlarge` and `c1.xlarge`) instances tend to be more
+available.
+- `--ebs-vol-size=GB` will attach an EBS volume with a given amount
+ of space to each node so that you can have a persistent HDFS cluster
+ on your nodes across cluster restarts (see below).
+- If one of your launches fails due to e.g. not having the right
+permissions on your private key file, you can run `launch` with the
+`--resume` option to restart the setup process on an existing cluster.
+
+# Running Jobs
+
+- Go into the `ec2` directory in the release of Spark you downloaded.
+- Run `./spark-ec2 -k <keypair> -i <key-file> login <cluster-name>` to
+ SSH into the cluster, where `<keypair>` and `<key-file>` are as
+ above. (This is just for convenience; you could also use
+ the EC2 console.)
+- To deploy code or data within your cluster, you can log in and use the
+ provided script `~/mesos-ec2/copy-dir`, which,
+ given a directory path, RSYNCs it to the same location on all the slaves.
+- If your job needs to access large datasets, the fastest way to do
+ that is to load them from Amazon S3 or an Amazon EBS device into an
+ instance of the Hadoop Distributed File System (HDFS) on your nodes.
+ The `spark-ec2` script already sets up a HDFS instance for you. It's
+ installed in `/root/ephemeral-hdfs`, and can be accessed using the
+ `bin/hadoop` script in that directory. Note that the data in this
+ HDFS goes away when you stop and restart a machine.
+- There is also a *persistent HDFS* instance in
+ `/root/presistent-hdfs` that will keep data across cluster restarts.
+ Typically each node has relatively little space of persistent data
+ (about 3 GB), but you can use the `--ebs-vol-size` option to
+ `spark-ec2` to attach a persistent EBS volume to each node for
+ storing the persistent HDFS.
+- Finally, if you get errors while running your jobs, look at the slave's logs
+ for that job using the Mesos web UI (`http://<master-hostname>:8080`).
+
+# Configuration
+
+You can edit `/root/spark/conf/spark-env.sh` on each machine to set Spark configuration options, such
+as JVM options and, most crucially, the amount of memory to use per machine (`SPARK_MEM`).
+This file needs to be copied to **every machine** to reflect the change. The easiest way to do this
+is to use a script we provide called `copy-dir`. First edit your `spark-env.sh` file on the master,
+then run `~/mesos-ec2/copy-dir /root/spark/conf` to RSYNC it to all the workers.
+
+The [configuration guide](configuration.html) describes the available configuration options.
+
+# Terminating a Cluster
+
+***Note that there is no way to recover data on EC2 nodes after shutting
+them down! Make sure you have copied everything important off the nodes
+before stopping them.***
+
+- Go into the `ec2` directory in the release of Spark you downloaded.
+- Run `./spark-ec2 destroy <cluster-name>`.
+
+# Pausing and Restarting Clusters
+
+The `spark-ec2` script also supports pausing a cluster. In this case,
+the VMs are stopped but not terminated, so they
+***lose all data on ephemeral disks*** but keep the data in their
+root partitions and their `persistent-hdfs`. Stopped machines will not
+cost you any EC2 cycles, but ***will*** continue to cost money for EBS
+storage.
+
+- To stop one of your clusters, go into the `ec2` directory and run
+`./spark-ec2 stop <cluster-name>`.
+- To restart it later, run
+`./spark-ec2 -i <key-file> start <cluster-name>`.
+- To ultimately destroy the cluster and stop consuming EBS space, run
+`./spark-ec2 destroy <cluster-name>` as described in the previous
+section.
+
+# Limitations
+
+- `spark-ec2` currently only launches machines in the US-East region of EC2.
+ It should not be hard to make it launch VMs in other zones, but you will need
+ to create your own AMIs in them.
+- Support for "cluster compute" nodes is limited -- there's no way to specify a
+ locality group. However, you can launch slave nodes in your
+ `<clusterName>-slaves` group manually and then use `spark-ec2 launch
+ --resume` to start a cluster with them.
+- Support for spot instances is limited.
+
+If you have a patch or suggestion for one of these limitations, feel free to
+[contribute](contributing-to-spark.html) it!
+
+# Using a Newer Spark Version
+
+The Spark EC2 machine images may not come with the latest version of Spark. To use a newer version, you can run `git pull` to pull in `/root/spark` to pull in the latest version of Spark from `git`, and build it using `sbt/sbt compile`. You will also need to copy it to all the other nodes in the cluster using `~/mesos-ec2/copy-dir /root/spark`.
+
+# Accessing Data in S3
+
+Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n://<id>:<secret>@<bucket>/path`, where `<id>` is your Amazon access key ID and `<secret>` is your Amazon secret access key. Note that you should escape any `/` characters in the secret key as `%2F`. Full instructions can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3).
+
+In addition to using a single input file, you can also use a directory of files as input by simply giving the path to the directory.
diff --git a/docs/img/glyphicons-halflings-white.png b/docs/img/glyphicons-halflings-white.png
new file mode 100755
index 0000000000..3bf6484a29
--- /dev/null
+++ b/docs/img/glyphicons-halflings-white.png
Binary files differ
diff --git a/docs/img/glyphicons-halflings.png b/docs/img/glyphicons-halflings.png
new file mode 100755
index 0000000000..a996999320
--- /dev/null
+++ b/docs/img/glyphicons-halflings.png
Binary files differ
diff --git a/docs/img/spark-logo-100x40px.png b/docs/img/spark-logo-100x40px.png
new file mode 100644
index 0000000000..54c3187bbd
--- /dev/null
+++ b/docs/img/spark-logo-100x40px.png
Binary files differ
diff --git a/docs/img/spark-logo-77x40px-hd.png b/docs/img/spark-logo-77x40px-hd.png
new file mode 100644
index 0000000000..270402f100
--- /dev/null
+++ b/docs/img/spark-logo-77x40px-hd.png
Binary files differ
diff --git a/docs/img/spark-logo-77x50px-hd.png b/docs/img/spark-logo-77x50px-hd.png
new file mode 100644
index 0000000000..6c5f0993c4
--- /dev/null
+++ b/docs/img/spark-logo-77x50px-hd.png
Binary files differ
diff --git a/docs/index.md b/docs/index.md
new file mode 100644
index 0000000000..ed9953a590
--- /dev/null
+++ b/docs/index.md
@@ -0,0 +1,96 @@
+---
+layout: global
+title: Spark Overview
+---
+
+{% comment %}
+TODO(andyk): Rewrite to make the Java API a first class part of the story.
+{% endcomment %}
+
+Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an
+interpreter. It provides clean, language-integrated APIs in Scala and Java, with a rich array of parallel operators. Spark can
+run on top of the [Apache Mesos](http://incubator.apache.org/mesos/) cluster manager,
+[Hadoop YARN](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html),
+Amazon EC2, or without an independent resource manager ("standalone mode").
+
+# Downloading
+
+Get Spark by visiting the [downloads page](http://spark-project.org/downloads.html) of the Spark website. This documentation is for Spark version {{site.SPARK_VERSION}}.
+
+# Building
+
+Spark requires [Scala {{site.SCALA_VERSION}}](http://www.scala-lang.org/). You will need to have Scala's `bin` directory in your `PATH`,
+or you will need to set the `SCALA_HOME` environment variable to point
+to where you've installed Scala. Scala must also be accessible through one
+of these methods on slave nodes on your cluster.
+
+Spark uses [Simple Build Tool](https://github.com/harrah/xsbt/wiki), which is bundled with it. To compile the code, go into the top-level Spark directory and run
+
+ sbt/sbt package
+
+# Testing the Build
+
+Spark comes with a number of sample programs in the `examples` directory.
+To run one of the samples, use `./run <class> <params>` in the top-level Spark directory
+(the `run` script sets up the appropriate paths and launches that program).
+For example, `./run spark.examples.SparkPi` will run a sample program that estimates Pi. Each of the
+examples prints usage help if no params are given.
+
+Note that all of the sample programs take a `<master>` parameter specifying the cluster URL
+to connect to. This can be a [URL for a distributed cluster](scala-programming-guide.html#master-urls),
+or `local` to run locally with one thread, or `local[N]` to run locally with N threads. You should start by using
+`local` for testing.
+
+Finally, Spark can be used interactively from a modified version of the Scala interpreter that you can start through
+`./spark-shell`. This is a great way to learn Spark.
+
+# A Note About Hadoop Versions
+
+Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported
+storage systems. Because the HDFS protocol has changed in different versions of
+Hadoop, you must build Spark against the same version that your cluster runs.
+You can change the version by setting the `HADOOP_VERSION` variable at the top
+of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`).
+
+# Where to Go from Here
+
+**Programming guides:**
+
+* [Quick Start](quick-start.html): a quick introduction to the Spark API; start here!
+* [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API
+* [Java Programming Guide](java-programming-guide.html): using Spark from Java
+
+**Deployment guides:**
+
+* [Running Spark on Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes
+* [Standalone Deploy Mode](spark-standalone.html): launch a standalone cluster quickly without a third-party cluster manager
+* [Running Spark on Mesos](running-on-mesos.html): deploy a private cluster using
+ [Apache Mesos](http://incubator.apache.org/mesos)
+* [Running Spark on YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
+
+**Other documents:**
+
+* [Configuration](configuration.html): customize Spark via its configuration system
+* [Tuning Guide](tuning.html): best practices to optimize performance and memory use
+* [API Docs (Scaladoc)](api/core/index.html)
+* [Bagel](bagel-programming-guide.html): an implementation of Google's Pregel on Spark
+* [Contributing to Spark](contributing-to-spark.html)
+
+**External resources:**
+
+* [Spark Homepage](http://www.spark-project.org)
+* [Mailing List](http://groups.google.com/group/spark-users): ask questions about Spark here
+* [AMP Camp](http://ampcamp.berkeley.edu/): a two-day training camp at UC Berkeley that featured talks and exercises
+ about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012),
+ [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are
+ available online for free.
+* [Code Examples](http://spark-project.org/examples.html): more are also available in the [examples subfolder](https://github.com/mesos/spark/tree/master/examples/src/main/scala/spark/examples) of Spark
+* [Paper Describing the Spark System](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf)
+
+# Community
+
+To get help using Spark or keep up with Spark development, sign up for the [spark-users mailing list](http://groups.google.com/group/spark-users).
+
+If you're in the San Francisco Bay Area, there's a regular [Spark meetup](http://www.meetup.com/spark-users/) every few weeks. Come by to meet the developers and other users.
+
+Finally, if you'd like to contribute code to Spark, read [how to contribute](contributing-to-spark.html).
diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md
new file mode 100644
index 0000000000..188ca4995e
--- /dev/null
+++ b/docs/java-programming-guide.md
@@ -0,0 +1,194 @@
+---
+layout: global
+title: Java Programming Guide
+---
+
+The Spark Java API exposes all the Spark features available in the Scala version to Java.
+To learn the basics of Spark, we recommend reading through the
+[Scala programming guide](scala-programming-guide.html) first; it should be
+easy to follow even if you don't know Scala.
+This guide will show how to use the Spark features described there in Java.
+
+The Spark Java API is defined in the
+[`spark.api.java`](api/core/index.html#spark.api.java.package) package, and includes
+a [`JavaSparkContext`](api/core/index.html#spark.api.java.JavaSparkContext) for
+initializing Spark and [`JavaRDD`](api/core/index.html#spark.api.java.JavaRDD) classes,
+which support the same methods as their Scala counterparts but take Java functions and return
+Java data and collection types. The main differences have to do with passing functions to RDD
+operations (e.g. map) and handling RDDs of different types, as discussed next.
+
+# Key Differences in the Java API
+
+There are a few key differences between the Java and Scala APIs:
+
+* Java does not support anonymous or first-class functions, so functions must
+ be implemented by extending the
+ [`spark.api.java.function.Function`](api/core/index.html#spark.api.java.function.Function),
+ [`Function2`](api/core/index.html#spark.api.java.function.Function2), etc.
+ classes.
+* To maintain type safety, the Java API defines specialized Function and RDD
+ classes for key-value pairs and doubles. For example,
+ [`JavaPairRDD`](api/core/index.html#spark.api.java.JavaPairRDD)
+ stores key-value pairs.
+* RDD methods like `collect()` and `countByKey()` return Java collections types,
+ such as `java.util.List` and `java.util.Map`.
+* Key-value pairs, which are simply written as `(key, value)` in Scala, are represented
+ by the `scala.Tuple2` class, and need to be created using `new Tuple2<K, V>(key, value)`.
+
+## RDD Classes
+
+Spark defines additional operations on RDDs of key-value pairs and doubles, such
+as `reduceByKey`, `join`, and `stdev`.
+
+In the Scala API, these methods are automatically added using Scala's
+[implicit conversions](http://www.scala-lang.org/node/130) mechanism.
+
+In the Java API, the extra methods are defined in the
+[`JavaPairRDD`](api/core/index.html#spark.api.java.JavaPairRDD)
+and [`JavaDoubleRDD`](api/core/index.html#spark.api.java.JavaDoubleRDD)
+classes. RDD methods like `map` are overloaded by specialized `PairFunction`
+and `DoubleFunction` classes, allowing them to return RDDs of the appropriate
+types. Common methods like `filter` and `sample` are implemented by
+each specialized RDD class, so filtering a `PairRDD` returns a new `PairRDD`,
+etc (this acheives the "same-result-type" principle used by the [Scala collections
+framework](http://docs.scala-lang.org/overviews/core/architecture-of-scala-collections.html)).
+
+## Function Classes
+
+The following table lists the function classes used by the Java API. Each
+class has a single abstract method, `call()`, that must be implemented.
+
+<table class="table">
+<tr><th>Class</th><th>Function Type</th></tr>
+
+<tr><td>Function&lt;T, R&gt;</td><td>T =&gt; R </td></tr>
+<tr><td>DoubleFunction&lt;T&gt;</td><td>T =&gt; Double </td></tr>
+<tr><td>PairFunction&lt;T, K, V&gt;</td><td>T =&gt; Tuple2&lt;K, V&gt; </td></tr>
+
+<tr><td>FlatMapFunction&lt;T, R&gt;</td><td>T =&gt; Iterable&lt;R&gt; </td></tr>
+<tr><td>DoubleFlatMapFunction&lt;T&gt;</td><td>T =&gt; Iterable&lt;Double&gt; </td></tr>
+<tr><td>PairFlatMapFunction&lt;T, K, V&gt;</td><td>T =&gt; Iterable&lt;Tuple2&lt;K, V&gt;&gt; </td></tr>
+
+<tr><td>Function2&lt;T1, T2, R&gt;</td><td>T1, T2 =&gt; R (function of two arguments)</td></tr>
+</table>
+
+## Storage Levels
+
+RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, such as `MEMORY_AND_DISK`, are
+declared in the [spark.api.java.StorageLevels](api/core/index.html#spark.api.java.StorageLevels) class.
+
+
+# Other Features
+
+The Java API supports other Spark features, including
+[accumulators](scala-programming-guide.html#accumulators),
+[broadcast variables](scala-programming-guide.html#broadcast-variables), and
+[caching](scala-programming-guide.html#rdd-persistence).
+
+
+# Example
+
+As an example, we will implement word count using the Java API.
+
+{% highlight java %}
+import spark.api.java.*;
+import spark.api.java.function.*;
+
+JavaSparkContext sc = new JavaSparkContext(...);
+JavaRDD<String> lines = ctx.textFile("hdfs://...");
+JavaRDD<String> words = lines.flatMap(
+ new FlatMapFunction<String, String>() {
+ public Iterable<String> call(String s) {
+ return Arrays.asList(s.split(" "));
+ }
+ }
+);
+{% endhighlight %}
+
+The word count program starts by creating a `JavaSparkContext`, which accepts
+the same parameters as its Scala counterpart. `JavaSparkContext` supports the
+same data loading methods as the regular `SparkContext`; here, `textFile`
+loads lines from text files stored in HDFS.
+
+To split the lines into words, we use `flatMap` to split each line on
+whitespace. `flatMap` is passed a `FlatMapFunction` that accepts a string and
+returns an `java.lang.Iterable` of strings.
+
+Here, the `FlatMapFunction` was created inline; another option is to subclass
+`FlatMapFunction` and pass an instance to `flatMap`:
+
+{% highlight java %}
+class Split extends FlatMapFunction<String, String> {
+ public Iterable<String> call(String s) {
+ return Arrays.asList(s.split(" "));
+ }
+);
+JavaRDD<String> words = lines.flatMap(new Split());
+{% endhighlight %}
+
+Continuing with the word count example, we map each word to a `(word, 1)` pair:
+
+{% highlight java %}
+import scala.Tuple2;
+JavaPairRDD<String, Integer> ones = words.map(
+ new PairFunction<String, String, Integer>() {
+ public Tuple2<String, Integer> call(String s) {
+ return new Tuple2(s, 1);
+ }
+ }
+);
+{% endhighlight %}
+
+Note that `map` was passed a `PairFunction<String, String, Integer>` and
+returned a `JavaPairRDD<String, Integer>`.
+
+To finish the word count program, we will use `reduceByKey` to count the
+occurrences of each word:
+
+{% highlight java %}
+JavaPairRDD<String, Integer> counts = ones.reduceByKey(
+ new Function2<Integer, Integer, Integer>() {
+ public Integer call(Integer i1, Integer i2) {
+ return i1 + i2;
+ }
+ }
+);
+{% endhighlight %}
+
+Here, `reduceByKey` is passed a `Function2`, which implements a function with
+two arguments. The resulting `JavaPairRDD` contains `(word, count)` pairs.
+
+In this example, we explicitly showed each intermediate RDD. It is also
+possible to chain the RDD transformations, so the word count example could also
+be written as:
+
+{% highlight java %}
+JavaPairRDD<String, Integer> counts = lines.flatMap(
+ ...
+ ).map(
+ ...
+ ).reduceByKey(
+ ...
+ );
+{% endhighlight %}
+
+There is no performance difference between these approaches; the choice is
+just a matter of style.
+
+# Javadoc
+
+We currently provide documentation for the Java API as Scaladoc, in the
+[`spark.api.java` package](api/core/index.html#spark.api.java.package), because
+some of the classes are implemented in Scala. The main downside is that the types and function
+definitions show Scala syntax (for example, `def reduce(func: Function2[T, T]): T` instead of
+`T reduce(Function2<T, T> func)`).
+We hope to generate documentation with Java-style syntax in the future.
+
+
+# Where to Go from Here
+
+Spark includes several sample programs using the Java API in
+`examples/src/main/java`. You can run them by passing the class name to the
+`run` script included in Spark -- for example, `./run
+spark.examples.JavaWordCount`. Each example program prints usage help when run
+without any arguments.
diff --git a/docs/js/main.js b/docs/js/main.js
new file mode 100755
index 0000000000..8b13789179
--- /dev/null
+++ b/docs/js/main.js
@@ -0,0 +1 @@
+
diff --git a/docs/js/vendor/bootstrap.js b/docs/js/vendor/bootstrap.js
new file mode 100755
index 0000000000..7f303eb88a
--- /dev/null
+++ b/docs/js/vendor/bootstrap.js
@@ -0,0 +1,2027 @@
+/* ===================================================
+ * bootstrap-transition.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#transitions
+ * ===================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ========================================================== */
+
+
+!function ($) {
+
+ $(function () {
+
+ "use strict"; // jshint ;_;
+
+
+ /* CSS TRANSITION SUPPORT (http://www.modernizr.com/)
+ * ======================================================= */
+
+ $.support.transition = (function () {
+
+ var transitionEnd = (function () {
+
+ var el = document.createElement('bootstrap')
+ , transEndEventNames = {
+ 'WebkitTransition' : 'webkitTransitionEnd'
+ , 'MozTransition' : 'transitionend'
+ , 'OTransition' : 'oTransitionEnd otransitionend'
+ , 'transition' : 'transitionend'
+ }
+ , name
+
+ for (name in transEndEventNames){
+ if (el.style[name] !== undefined) {
+ return transEndEventNames[name]
+ }
+ }
+
+ }())
+
+ return transitionEnd && {
+ end: transitionEnd
+ }
+
+ })()
+
+ })
+
+}(window.jQuery);/* ==========================================================
+ * bootstrap-alert.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#alerts
+ * ==========================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ========================================================== */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* ALERT CLASS DEFINITION
+ * ====================== */
+
+ var dismiss = '[data-dismiss="alert"]'
+ , Alert = function (el) {
+ $(el).on('click', dismiss, this.close)
+ }
+
+ Alert.prototype.close = function (e) {
+ var $this = $(this)
+ , selector = $this.attr('data-target')
+ , $parent
+
+ if (!selector) {
+ selector = $this.attr('href')
+ selector = selector && selector.replace(/.*(?=#[^\s]*$)/, '') //strip for ie7
+ }
+
+ $parent = $(selector)
+
+ e && e.preventDefault()
+
+ $parent.length || ($parent = $this.hasClass('alert') ? $this : $this.parent())
+
+ $parent.trigger(e = $.Event('close'))
+
+ if (e.isDefaultPrevented()) return
+
+ $parent.removeClass('in')
+
+ function removeElement() {
+ $parent
+ .trigger('closed')
+ .remove()
+ }
+
+ $.support.transition && $parent.hasClass('fade') ?
+ $parent.on($.support.transition.end, removeElement) :
+ removeElement()
+ }
+
+
+ /* ALERT PLUGIN DEFINITION
+ * ======================= */
+
+ $.fn.alert = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('alert')
+ if (!data) $this.data('alert', (data = new Alert(this)))
+ if (typeof option == 'string') data[option].call($this)
+ })
+ }
+
+ $.fn.alert.Constructor = Alert
+
+
+ /* ALERT DATA-API
+ * ============== */
+
+ $(function () {
+ $('body').on('click.alert.data-api', dismiss, Alert.prototype.close)
+ })
+
+}(window.jQuery);/* ============================================================
+ * bootstrap-button.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#buttons
+ * ============================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ============================================================ */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* BUTTON PUBLIC CLASS DEFINITION
+ * ============================== */
+
+ var Button = function (element, options) {
+ this.$element = $(element)
+ this.options = $.extend({}, $.fn.button.defaults, options)
+ }
+
+ Button.prototype.setState = function (state) {
+ var d = 'disabled'
+ , $el = this.$element
+ , data = $el.data()
+ , val = $el.is('input') ? 'val' : 'html'
+
+ state = state + 'Text'
+ data.resetText || $el.data('resetText', $el[val]())
+
+ $el[val](data[state] || this.options[state])
+
+ // push to event loop to allow forms to submit
+ setTimeout(function () {
+ state == 'loadingText' ?
+ $el.addClass(d).attr(d, d) :
+ $el.removeClass(d).removeAttr(d)
+ }, 0)
+ }
+
+ Button.prototype.toggle = function () {
+ var $parent = this.$element.parent('[data-toggle="buttons-radio"]')
+
+ $parent && $parent
+ .find('.active')
+ .removeClass('active')
+
+ this.$element.toggleClass('active')
+ }
+
+
+ /* BUTTON PLUGIN DEFINITION
+ * ======================== */
+
+ $.fn.button = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('button')
+ , options = typeof option == 'object' && option
+ if (!data) $this.data('button', (data = new Button(this, options)))
+ if (option == 'toggle') data.toggle()
+ else if (option) data.setState(option)
+ })
+ }
+
+ $.fn.button.defaults = {
+ loadingText: 'loading...'
+ }
+
+ $.fn.button.Constructor = Button
+
+
+ /* BUTTON DATA-API
+ * =============== */
+
+ $(function () {
+ $('body').on('click.button.data-api', '[data-toggle^=button]', function ( e ) {
+ var $btn = $(e.target)
+ if (!$btn.hasClass('btn')) $btn = $btn.closest('.btn')
+ $btn.button('toggle')
+ })
+ })
+
+}(window.jQuery);/* ==========================================================
+ * bootstrap-carousel.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#carousel
+ * ==========================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ========================================================== */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* CAROUSEL CLASS DEFINITION
+ * ========================= */
+
+ var Carousel = function (element, options) {
+ this.$element = $(element)
+ this.options = options
+ this.options.slide && this.slide(this.options.slide)
+ this.options.pause == 'hover' && this.$element
+ .on('mouseenter', $.proxy(this.pause, this))
+ .on('mouseleave', $.proxy(this.cycle, this))
+ }
+
+ Carousel.prototype = {
+
+ cycle: function (e) {
+ if (!e) this.paused = false
+ this.options.interval
+ && !this.paused
+ && (this.interval = setInterval($.proxy(this.next, this), this.options.interval))
+ return this
+ }
+
+ , to: function (pos) {
+ var $active = this.$element.find('.item.active')
+ , children = $active.parent().children()
+ , activePos = children.index($active)
+ , that = this
+
+ if (pos > (children.length - 1) || pos < 0) return
+
+ if (this.sliding) {
+ return this.$element.one('slid', function () {
+ that.to(pos)
+ })
+ }
+
+ if (activePos == pos) {
+ return this.pause().cycle()
+ }
+
+ return this.slide(pos > activePos ? 'next' : 'prev', $(children[pos]))
+ }
+
+ , pause: function (e) {
+ if (!e) this.paused = true
+ if (this.$element.find('.next, .prev').length && $.support.transition.end) {
+ this.$element.trigger($.support.transition.end)
+ this.cycle()
+ }
+ clearInterval(this.interval)
+ this.interval = null
+ return this
+ }
+
+ , next: function () {
+ if (this.sliding) return
+ return this.slide('next')
+ }
+
+ , prev: function () {
+ if (this.sliding) return
+ return this.slide('prev')
+ }
+
+ , slide: function (type, next) {
+ var $active = this.$element.find('.item.active')
+ , $next = next || $active[type]()
+ , isCycling = this.interval
+ , direction = type == 'next' ? 'left' : 'right'
+ , fallback = type == 'next' ? 'first' : 'last'
+ , that = this
+ , e = $.Event('slide', {
+ relatedTarget: $next[0]
+ })
+
+ this.sliding = true
+
+ isCycling && this.pause()
+
+ $next = $next.length ? $next : this.$element.find('.item')[fallback]()
+
+ if ($next.hasClass('active')) return
+
+ if ($.support.transition && this.$element.hasClass('slide')) {
+ this.$element.trigger(e)
+ if (e.isDefaultPrevented()) return
+ $next.addClass(type)
+ $next[0].offsetWidth // force reflow
+ $active.addClass(direction)
+ $next.addClass(direction)
+ this.$element.one($.support.transition.end, function () {
+ $next.removeClass([type, direction].join(' ')).addClass('active')
+ $active.removeClass(['active', direction].join(' '))
+ that.sliding = false
+ setTimeout(function () { that.$element.trigger('slid') }, 0)
+ })
+ } else {
+ this.$element.trigger(e)
+ if (e.isDefaultPrevented()) return
+ $active.removeClass('active')
+ $next.addClass('active')
+ this.sliding = false
+ this.$element.trigger('slid')
+ }
+
+ isCycling && this.cycle()
+
+ return this
+ }
+
+ }
+
+
+ /* CAROUSEL PLUGIN DEFINITION
+ * ========================== */
+
+ $.fn.carousel = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('carousel')
+ , options = $.extend({}, $.fn.carousel.defaults, typeof option == 'object' && option)
+ , action = typeof option == 'string' ? option : options.slide
+ if (!data) $this.data('carousel', (data = new Carousel(this, options)))
+ if (typeof option == 'number') data.to(option)
+ else if (action) data[action]()
+ else if (options.interval) data.cycle()
+ })
+ }
+
+ $.fn.carousel.defaults = {
+ interval: 5000
+ , pause: 'hover'
+ }
+
+ $.fn.carousel.Constructor = Carousel
+
+
+ /* CAROUSEL DATA-API
+ * ================= */
+
+ $(function () {
+ $('body').on('click.carousel.data-api', '[data-slide]', function ( e ) {
+ var $this = $(this), href
+ , $target = $($this.attr('data-target') || (href = $this.attr('href')) && href.replace(/.*(?=#[^\s]+$)/, '')) //strip for ie7
+ , options = !$target.data('modal') && $.extend({}, $target.data(), $this.data())
+ $target.carousel(options)
+ e.preventDefault()
+ })
+ })
+
+}(window.jQuery);/* =============================================================
+ * bootstrap-collapse.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#collapse
+ * =============================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ============================================================ */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* COLLAPSE PUBLIC CLASS DEFINITION
+ * ================================ */
+
+ var Collapse = function (element, options) {
+ this.$element = $(element)
+ this.options = $.extend({}, $.fn.collapse.defaults, options)
+
+ if (this.options.parent) {
+ this.$parent = $(this.options.parent)
+ }
+
+ this.options.toggle && this.toggle()
+ }
+
+ Collapse.prototype = {
+
+ constructor: Collapse
+
+ , dimension: function () {
+ var hasWidth = this.$element.hasClass('width')
+ return hasWidth ? 'width' : 'height'
+ }
+
+ , show: function () {
+ var dimension
+ , scroll
+ , actives
+ , hasData
+
+ if (this.transitioning) return
+
+ dimension = this.dimension()
+ scroll = $.camelCase(['scroll', dimension].join('-'))
+ actives = this.$parent && this.$parent.find('> .accordion-group > .in')
+
+ if (actives && actives.length) {
+ hasData = actives.data('collapse')
+ if (hasData && hasData.transitioning) return
+ actives.collapse('hide')
+ hasData || actives.data('collapse', null)
+ }
+
+ this.$element[dimension](0)
+ this.transition('addClass', $.Event('show'), 'shown')
+ $.support.transition && this.$element[dimension](this.$element[0][scroll])
+ }
+
+ , hide: function () {
+ var dimension
+ if (this.transitioning) return
+ dimension = this.dimension()
+ this.reset(this.$element[dimension]())
+ this.transition('removeClass', $.Event('hide'), 'hidden')
+ this.$element[dimension](0)
+ }
+
+ , reset: function (size) {
+ var dimension = this.dimension()
+
+ this.$element
+ .removeClass('collapse')
+ [dimension](size || 'auto')
+ [0].offsetWidth
+
+ this.$element[size !== null ? 'addClass' : 'removeClass']('collapse')
+
+ return this
+ }
+
+ , transition: function (method, startEvent, completeEvent) {
+ var that = this
+ , complete = function () {
+ if (startEvent.type == 'show') that.reset()
+ that.transitioning = 0
+ that.$element.trigger(completeEvent)
+ }
+
+ this.$element.trigger(startEvent)
+
+ if (startEvent.isDefaultPrevented()) return
+
+ this.transitioning = 1
+
+ this.$element[method]('in')
+
+ $.support.transition && this.$element.hasClass('collapse') ?
+ this.$element.one($.support.transition.end, complete) :
+ complete()
+ }
+
+ , toggle: function () {
+ this[this.$element.hasClass('in') ? 'hide' : 'show']()
+ }
+
+ }
+
+
+ /* COLLAPSIBLE PLUGIN DEFINITION
+ * ============================== */
+
+ $.fn.collapse = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('collapse')
+ , options = typeof option == 'object' && option
+ if (!data) $this.data('collapse', (data = new Collapse(this, options)))
+ if (typeof option == 'string') data[option]()
+ })
+ }
+
+ $.fn.collapse.defaults = {
+ toggle: true
+ }
+
+ $.fn.collapse.Constructor = Collapse
+
+
+ /* COLLAPSIBLE DATA-API
+ * ==================== */
+
+ $(function () {
+ $('body').on('click.collapse.data-api', '[data-toggle=collapse]', function (e) {
+ var $this = $(this), href
+ , target = $this.attr('data-target')
+ || e.preventDefault()
+ || (href = $this.attr('href')) && href.replace(/.*(?=#[^\s]+$)/, '') //strip for ie7
+ , option = $(target).data('collapse') ? 'toggle' : $this.data()
+ $this[$(target).hasClass('in') ? 'addClass' : 'removeClass']('collapsed')
+ $(target).collapse(option)
+ })
+ })
+
+}(window.jQuery);/* ============================================================
+ * bootstrap-dropdown.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#dropdowns
+ * ============================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ============================================================ */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* DROPDOWN CLASS DEFINITION
+ * ========================= */
+
+ var toggle = '[data-toggle=dropdown]'
+ , Dropdown = function (element) {
+ var $el = $(element).on('click.dropdown.data-api', this.toggle)
+ $('html').on('click.dropdown.data-api', function () {
+ $el.parent().removeClass('open')
+ })
+ }
+
+ Dropdown.prototype = {
+
+ constructor: Dropdown
+
+ , toggle: function (e) {
+ var $this = $(this)
+ , $parent
+ , isActive
+
+ if ($this.is('.disabled, :disabled')) return
+
+ $parent = getParent($this)
+
+ isActive = $parent.hasClass('open')
+
+ clearMenus()
+
+ if (!isActive) {
+ $parent.toggleClass('open')
+ $this.focus()
+ }
+
+ return false
+ }
+
+ , keydown: function (e) {
+ var $this
+ , $items
+ , $active
+ , $parent
+ , isActive
+ , index
+
+ if (!/(38|40|27)/.test(e.keyCode)) return
+
+ $this = $(this)
+
+ e.preventDefault()
+ e.stopPropagation()
+
+ if ($this.is('.disabled, :disabled')) return
+
+ $parent = getParent($this)
+
+ isActive = $parent.hasClass('open')
+
+ if (!isActive || (isActive && e.keyCode == 27)) return $this.click()
+
+ $items = $('[role=menu] li:not(.divider) a', $parent)
+
+ if (!$items.length) return
+
+ index = $items.index($items.filter(':focus'))
+
+ if (e.keyCode == 38 && index > 0) index-- // up
+ if (e.keyCode == 40 && index < $items.length - 1) index++ // down
+ if (!~index) index = 0
+
+ $items
+ .eq(index)
+ .focus()
+ }
+
+ }
+
+ function clearMenus() {
+ getParent($(toggle))
+ .removeClass('open')
+ }
+
+ function getParent($this) {
+ var selector = $this.attr('data-target')
+ , $parent
+
+ if (!selector) {
+ selector = $this.attr('href')
+ selector = selector && selector.replace(/.*(?=#[^\s]*$)/, '') //strip for ie7
+ }
+
+ $parent = $(selector)
+ $parent.length || ($parent = $this.parent())
+
+ return $parent
+ }
+
+
+ /* DROPDOWN PLUGIN DEFINITION
+ * ========================== */
+
+ $.fn.dropdown = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('dropdown')
+ if (!data) $this.data('dropdown', (data = new Dropdown(this)))
+ if (typeof option == 'string') data[option].call($this)
+ })
+ }
+
+ $.fn.dropdown.Constructor = Dropdown
+
+
+ /* APPLY TO STANDARD DROPDOWN ELEMENTS
+ * =================================== */
+
+ $(function () {
+ $('html')
+ .on('click.dropdown.data-api touchstart.dropdown.data-api', clearMenus)
+ $('body')
+ .on('click.dropdown touchstart.dropdown.data-api', '.dropdown', function (e) { e.stopPropagation() })
+ .on('click.dropdown.data-api touchstart.dropdown.data-api' , toggle, Dropdown.prototype.toggle)
+ .on('keydown.dropdown.data-api touchstart.dropdown.data-api', toggle + ', [role=menu]' , Dropdown.prototype.keydown)
+ })
+
+}(window.jQuery);/* =========================================================
+ * bootstrap-modal.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#modals
+ * =========================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ========================================================= */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* MODAL CLASS DEFINITION
+ * ====================== */
+
+ var Modal = function (element, options) {
+ this.options = options
+ this.$element = $(element)
+ .delegate('[data-dismiss="modal"]', 'click.dismiss.modal', $.proxy(this.hide, this))
+ this.options.remote && this.$element.find('.modal-body').load(this.options.remote)
+ }
+
+ Modal.prototype = {
+
+ constructor: Modal
+
+ , toggle: function () {
+ return this[!this.isShown ? 'show' : 'hide']()
+ }
+
+ , show: function () {
+ var that = this
+ , e = $.Event('show')
+
+ this.$element.trigger(e)
+
+ if (this.isShown || e.isDefaultPrevented()) return
+
+ $('body').addClass('modal-open')
+
+ this.isShown = true
+
+ this.escape()
+
+ this.backdrop(function () {
+ var transition = $.support.transition && that.$element.hasClass('fade')
+
+ if (!that.$element.parent().length) {
+ that.$element.appendTo(document.body) //don't move modals dom position
+ }
+
+ that.$element
+ .show()
+
+ if (transition) {
+ that.$element[0].offsetWidth // force reflow
+ }
+
+ that.$element
+ .addClass('in')
+ .attr('aria-hidden', false)
+ .focus()
+
+ that.enforceFocus()
+
+ transition ?
+ that.$element.one($.support.transition.end, function () { that.$element.trigger('shown') }) :
+ that.$element.trigger('shown')
+
+ })
+ }
+
+ , hide: function (e) {
+ e && e.preventDefault()
+
+ var that = this
+
+ e = $.Event('hide')
+
+ this.$element.trigger(e)
+
+ if (!this.isShown || e.isDefaultPrevented()) return
+
+ this.isShown = false
+
+ $('body').removeClass('modal-open')
+
+ this.escape()
+
+ $(document).off('focusin.modal')
+
+ this.$element
+ .removeClass('in')
+ .attr('aria-hidden', true)
+
+ $.support.transition && this.$element.hasClass('fade') ?
+ this.hideWithTransition() :
+ this.hideModal()
+ }
+
+ , enforceFocus: function () {
+ var that = this
+ $(document).on('focusin.modal', function (e) {
+ if (that.$element[0] !== e.target && !that.$element.has(e.target).length) {
+ that.$element.focus()
+ }
+ })
+ }
+
+ , escape: function () {
+ var that = this
+ if (this.isShown && this.options.keyboard) {
+ this.$element.on('keyup.dismiss.modal', function ( e ) {
+ e.which == 27 && that.hide()
+ })
+ } else if (!this.isShown) {
+ this.$element.off('keyup.dismiss.modal')
+ }
+ }
+
+ , hideWithTransition: function () {
+ var that = this
+ , timeout = setTimeout(function () {
+ that.$element.off($.support.transition.end)
+ that.hideModal()
+ }, 500)
+
+ this.$element.one($.support.transition.end, function () {
+ clearTimeout(timeout)
+ that.hideModal()
+ })
+ }
+
+ , hideModal: function (that) {
+ this.$element
+ .hide()
+ .trigger('hidden')
+
+ this.backdrop()
+ }
+
+ , removeBackdrop: function () {
+ this.$backdrop.remove()
+ this.$backdrop = null
+ }
+
+ , backdrop: function (callback) {
+ var that = this
+ , animate = this.$element.hasClass('fade') ? 'fade' : ''
+
+ if (this.isShown && this.options.backdrop) {
+ var doAnimate = $.support.transition && animate
+
+ this.$backdrop = $('<div class="modal-backdrop ' + animate + '" />')
+ .appendTo(document.body)
+
+ if (this.options.backdrop != 'static') {
+ this.$backdrop.click($.proxy(this.hide, this))
+ }
+
+ if (doAnimate) this.$backdrop[0].offsetWidth // force reflow
+
+ this.$backdrop.addClass('in')
+
+ doAnimate ?
+ this.$backdrop.one($.support.transition.end, callback) :
+ callback()
+
+ } else if (!this.isShown && this.$backdrop) {
+ this.$backdrop.removeClass('in')
+
+ $.support.transition && this.$element.hasClass('fade')?
+ this.$backdrop.one($.support.transition.end, $.proxy(this.removeBackdrop, this)) :
+ this.removeBackdrop()
+
+ } else if (callback) {
+ callback()
+ }
+ }
+ }
+
+
+ /* MODAL PLUGIN DEFINITION
+ * ======================= */
+
+ $.fn.modal = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('modal')
+ , options = $.extend({}, $.fn.modal.defaults, $this.data(), typeof option == 'object' && option)
+ if (!data) $this.data('modal', (data = new Modal(this, options)))
+ if (typeof option == 'string') data[option]()
+ else if (options.show) data.show()
+ })
+ }
+
+ $.fn.modal.defaults = {
+ backdrop: true
+ , keyboard: true
+ , show: true
+ }
+
+ $.fn.modal.Constructor = Modal
+
+
+ /* MODAL DATA-API
+ * ============== */
+
+ $(function () {
+ $('body').on('click.modal.data-api', '[data-toggle="modal"]', function ( e ) {
+ var $this = $(this)
+ , href = $this.attr('href')
+ , $target = $($this.attr('data-target') || (href && href.replace(/.*(?=#[^\s]+$)/, ''))) //strip for ie7
+ , option = $target.data('modal') ? 'toggle' : $.extend({ remote: !/#/.test(href) && href }, $target.data(), $this.data())
+
+ e.preventDefault()
+
+ $target
+ .modal(option)
+ .one('hide', function () {
+ $this.focus()
+ })
+ })
+ })
+
+}(window.jQuery);/* ===========================================================
+ * bootstrap-tooltip.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#tooltips
+ * Inspired by the original jQuery.tipsy by Jason Frame
+ * ===========================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ========================================================== */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* TOOLTIP PUBLIC CLASS DEFINITION
+ * =============================== */
+
+ var Tooltip = function (element, options) {
+ this.init('tooltip', element, options)
+ }
+
+ Tooltip.prototype = {
+
+ constructor: Tooltip
+
+ , init: function (type, element, options) {
+ var eventIn
+ , eventOut
+
+ this.type = type
+ this.$element = $(element)
+ this.options = this.getOptions(options)
+ this.enabled = true
+
+ if (this.options.trigger == 'click') {
+ this.$element.on('click.' + this.type, this.options.selector, $.proxy(this.toggle, this))
+ } else if (this.options.trigger != 'manual') {
+ eventIn = this.options.trigger == 'hover' ? 'mouseenter' : 'focus'
+ eventOut = this.options.trigger == 'hover' ? 'mouseleave' : 'blur'
+ this.$element.on(eventIn + '.' + this.type, this.options.selector, $.proxy(this.enter, this))
+ this.$element.on(eventOut + '.' + this.type, this.options.selector, $.proxy(this.leave, this))
+ }
+
+ this.options.selector ?
+ (this._options = $.extend({}, this.options, { trigger: 'manual', selector: '' })) :
+ this.fixTitle()
+ }
+
+ , getOptions: function (options) {
+ options = $.extend({}, $.fn[this.type].defaults, options, this.$element.data())
+
+ if (options.delay && typeof options.delay == 'number') {
+ options.delay = {
+ show: options.delay
+ , hide: options.delay
+ }
+ }
+
+ return options
+ }
+
+ , enter: function (e) {
+ var self = $(e.currentTarget)[this.type](this._options).data(this.type)
+
+ if (!self.options.delay || !self.options.delay.show) return self.show()
+
+ clearTimeout(this.timeout)
+ self.hoverState = 'in'
+ this.timeout = setTimeout(function() {
+ if (self.hoverState == 'in') self.show()
+ }, self.options.delay.show)
+ }
+
+ , leave: function (e) {
+ var self = $(e.currentTarget)[this.type](this._options).data(this.type)
+
+ if (this.timeout) clearTimeout(this.timeout)
+ if (!self.options.delay || !self.options.delay.hide) return self.hide()
+
+ self.hoverState = 'out'
+ this.timeout = setTimeout(function() {
+ if (self.hoverState == 'out') self.hide()
+ }, self.options.delay.hide)
+ }
+
+ , show: function () {
+ var $tip
+ , inside
+ , pos
+ , actualWidth
+ , actualHeight
+ , placement
+ , tp
+
+ if (this.hasContent() && this.enabled) {
+ $tip = this.tip()
+ this.setContent()
+
+ if (this.options.animation) {
+ $tip.addClass('fade')
+ }
+
+ placement = typeof this.options.placement == 'function' ?
+ this.options.placement.call(this, $tip[0], this.$element[0]) :
+ this.options.placement
+
+ inside = /in/.test(placement)
+
+ $tip
+ .remove()
+ .css({ top: 0, left: 0, display: 'block' })
+ .appendTo(inside ? this.$element : document.body)
+
+ pos = this.getPosition(inside)
+
+ actualWidth = $tip[0].offsetWidth
+ actualHeight = $tip[0].offsetHeight
+
+ switch (inside ? placement.split(' ')[1] : placement) {
+ case 'bottom':
+ tp = {top: pos.top + pos.height, left: pos.left + pos.width / 2 - actualWidth / 2}
+ break
+ case 'top':
+ tp = {top: pos.top - actualHeight, left: pos.left + pos.width / 2 - actualWidth / 2}
+ break
+ case 'left':
+ tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left - actualWidth}
+ break
+ case 'right':
+ tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left + pos.width}
+ break
+ }
+
+ $tip
+ .css(tp)
+ .addClass(placement)
+ .addClass('in')
+ }
+ }
+
+ , setContent: function () {
+ var $tip = this.tip()
+ , title = this.getTitle()
+
+ $tip.find('.tooltip-inner')[this.options.html ? 'html' : 'text'](title)
+ $tip.removeClass('fade in top bottom left right')
+ }
+
+ , hide: function () {
+ var that = this
+ , $tip = this.tip()
+
+ $tip.removeClass('in')
+
+ function removeWithAnimation() {
+ var timeout = setTimeout(function () {
+ $tip.off($.support.transition.end).remove()
+ }, 500)
+
+ $tip.one($.support.transition.end, function () {
+ clearTimeout(timeout)
+ $tip.remove()
+ })
+ }
+
+ $.support.transition && this.$tip.hasClass('fade') ?
+ removeWithAnimation() :
+ $tip.remove()
+
+ return this
+ }
+
+ , fixTitle: function () {
+ var $e = this.$element
+ if ($e.attr('title') || typeof($e.attr('data-original-title')) != 'string') {
+ $e.attr('data-original-title', $e.attr('title') || '').removeAttr('title')
+ }
+ }
+
+ , hasContent: function () {
+ return this.getTitle()
+ }
+
+ , getPosition: function (inside) {
+ return $.extend({}, (inside ? {top: 0, left: 0} : this.$element.offset()), {
+ width: this.$element[0].offsetWidth
+ , height: this.$element[0].offsetHeight
+ })
+ }
+
+ , getTitle: function () {
+ var title
+ , $e = this.$element
+ , o = this.options
+
+ title = $e.attr('data-original-title')
+ || (typeof o.title == 'function' ? o.title.call($e[0]) : o.title)
+
+ return title
+ }
+
+ , tip: function () {
+ return this.$tip = this.$tip || $(this.options.template)
+ }
+
+ , validate: function () {
+ if (!this.$element[0].parentNode) {
+ this.hide()
+ this.$element = null
+ this.options = null
+ }
+ }
+
+ , enable: function () {
+ this.enabled = true
+ }
+
+ , disable: function () {
+ this.enabled = false
+ }
+
+ , toggleEnabled: function () {
+ this.enabled = !this.enabled
+ }
+
+ , toggle: function () {
+ this[this.tip().hasClass('in') ? 'hide' : 'show']()
+ }
+
+ , destroy: function () {
+ this.hide().$element.off('.' + this.type).removeData(this.type)
+ }
+
+ }
+
+
+ /* TOOLTIP PLUGIN DEFINITION
+ * ========================= */
+
+ $.fn.tooltip = function ( option ) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('tooltip')
+ , options = typeof option == 'object' && option
+ if (!data) $this.data('tooltip', (data = new Tooltip(this, options)))
+ if (typeof option == 'string') data[option]()
+ })
+ }
+
+ $.fn.tooltip.Constructor = Tooltip
+
+ $.fn.tooltip.defaults = {
+ animation: true
+ , placement: 'top'
+ , selector: false
+ , template: '<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>'
+ , trigger: 'hover'
+ , title: ''
+ , delay: 0
+ , html: true
+ }
+
+}(window.jQuery);
+/* ===========================================================
+ * bootstrap-popover.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#popovers
+ * ===========================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * =========================================================== */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* POPOVER PUBLIC CLASS DEFINITION
+ * =============================== */
+
+ var Popover = function (element, options) {
+ this.init('popover', element, options)
+ }
+
+
+ /* NOTE: POPOVER EXTENDS BOOTSTRAP-TOOLTIP.js
+ ========================================== */
+
+ Popover.prototype = $.extend({}, $.fn.tooltip.Constructor.prototype, {
+
+ constructor: Popover
+
+ , setContent: function () {
+ var $tip = this.tip()
+ , title = this.getTitle()
+ , content = this.getContent()
+
+ $tip.find('.popover-title')[this.options.html ? 'html' : 'text'](title)
+ $tip.find('.popover-content > *')[this.options.html ? 'html' : 'text'](content)
+
+ $tip.removeClass('fade top bottom left right in')
+ }
+
+ , hasContent: function () {
+ return this.getTitle() || this.getContent()
+ }
+
+ , getContent: function () {
+ var content
+ , $e = this.$element
+ , o = this.options
+
+ content = $e.attr('data-content')
+ || (typeof o.content == 'function' ? o.content.call($e[0]) : o.content)
+
+ return content
+ }
+
+ , tip: function () {
+ if (!this.$tip) {
+ this.$tip = $(this.options.template)
+ }
+ return this.$tip
+ }
+
+ , destroy: function () {
+ this.hide().$element.off('.' + this.type).removeData(this.type)
+ }
+
+ })
+
+
+ /* POPOVER PLUGIN DEFINITION
+ * ======================= */
+
+ $.fn.popover = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('popover')
+ , options = typeof option == 'object' && option
+ if (!data) $this.data('popover', (data = new Popover(this, options)))
+ if (typeof option == 'string') data[option]()
+ })
+ }
+
+ $.fn.popover.Constructor = Popover
+
+ $.fn.popover.defaults = $.extend({} , $.fn.tooltip.defaults, {
+ placement: 'right'
+ , trigger: 'click'
+ , content: ''
+ , template: '<div class="popover"><div class="arrow"></div><div class="popover-inner"><h3 class="popover-title"></h3><div class="popover-content"><p></p></div></div></div>'
+ })
+
+}(window.jQuery);/* =============================================================
+ * bootstrap-scrollspy.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#scrollspy
+ * =============================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ============================================================== */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* SCROLLSPY CLASS DEFINITION
+ * ========================== */
+
+ function ScrollSpy(element, options) {
+ var process = $.proxy(this.process, this)
+ , $element = $(element).is('body') ? $(window) : $(element)
+ , href
+ this.options = $.extend({}, $.fn.scrollspy.defaults, options)
+ this.$scrollElement = $element.on('scroll.scroll-spy.data-api', process)
+ this.selector = (this.options.target
+ || ((href = $(element).attr('href')) && href.replace(/.*(?=#[^\s]+$)/, '')) //strip for ie7
+ || '') + ' .nav li > a'
+ this.$body = $('body')
+ this.refresh()
+ this.process()
+ }
+
+ ScrollSpy.prototype = {
+
+ constructor: ScrollSpy
+
+ , refresh: function () {
+ var self = this
+ , $targets
+
+ this.offsets = $([])
+ this.targets = $([])
+
+ $targets = this.$body
+ .find(this.selector)
+ .map(function () {
+ var $el = $(this)
+ , href = $el.data('target') || $el.attr('href')
+ , $href = /^#\w/.test(href) && $(href)
+ return ( $href
+ && $href.length
+ && [[ $href.position().top, href ]] ) || null
+ })
+ .sort(function (a, b) { return a[0] - b[0] })
+ .each(function () {
+ self.offsets.push(this[0])
+ self.targets.push(this[1])
+ })
+ }
+
+ , process: function () {
+ var scrollTop = this.$scrollElement.scrollTop() + this.options.offset
+ , scrollHeight = this.$scrollElement[0].scrollHeight || this.$body[0].scrollHeight
+ , maxScroll = scrollHeight - this.$scrollElement.height()
+ , offsets = this.offsets
+ , targets = this.targets
+ , activeTarget = this.activeTarget
+ , i
+
+ if (scrollTop >= maxScroll) {
+ return activeTarget != (i = targets.last()[0])
+ && this.activate ( i )
+ }
+
+ for (i = offsets.length; i--;) {
+ activeTarget != targets[i]
+ && scrollTop >= offsets[i]
+ && (!offsets[i + 1] || scrollTop <= offsets[i + 1])
+ && this.activate( targets[i] )
+ }
+ }
+
+ , activate: function (target) {
+ var active
+ , selector
+
+ this.activeTarget = target
+
+ $(this.selector)
+ .parent('.active')
+ .removeClass('active')
+
+ selector = this.selector
+ + '[data-target="' + target + '"],'
+ + this.selector + '[href="' + target + '"]'
+
+ active = $(selector)
+ .parent('li')
+ .addClass('active')
+
+ if (active.parent('.dropdown-menu').length) {
+ active = active.closest('li.dropdown').addClass('active')
+ }
+
+ active.trigger('activate')
+ }
+
+ }
+
+
+ /* SCROLLSPY PLUGIN DEFINITION
+ * =========================== */
+
+ $.fn.scrollspy = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('scrollspy')
+ , options = typeof option == 'object' && option
+ if (!data) $this.data('scrollspy', (data = new ScrollSpy(this, options)))
+ if (typeof option == 'string') data[option]()
+ })
+ }
+
+ $.fn.scrollspy.Constructor = ScrollSpy
+
+ $.fn.scrollspy.defaults = {
+ offset: 10
+ }
+
+
+ /* SCROLLSPY DATA-API
+ * ================== */
+
+ $(window).on('load', function () {
+ $('[data-spy="scroll"]').each(function () {
+ var $spy = $(this)
+ $spy.scrollspy($spy.data())
+ })
+ })
+
+}(window.jQuery);/* ========================================================
+ * bootstrap-tab.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#tabs
+ * ========================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ======================================================== */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* TAB CLASS DEFINITION
+ * ==================== */
+
+ var Tab = function (element) {
+ this.element = $(element)
+ }
+
+ Tab.prototype = {
+
+ constructor: Tab
+
+ , show: function () {
+ var $this = this.element
+ , $ul = $this.closest('ul:not(.dropdown-menu)')
+ , selector = $this.attr('data-target')
+ , previous
+ , $target
+ , e
+
+ if (!selector) {
+ selector = $this.attr('href')
+ selector = selector && selector.replace(/.*(?=#[^\s]*$)/, '') //strip for ie7
+ }
+
+ if ( $this.parent('li').hasClass('active') ) return
+
+ previous = $ul.find('.active a').last()[0]
+
+ e = $.Event('show', {
+ relatedTarget: previous
+ })
+
+ $this.trigger(e)
+
+ if (e.isDefaultPrevented()) return
+
+ $target = $(selector)
+
+ this.activate($this.parent('li'), $ul)
+ this.activate($target, $target.parent(), function () {
+ $this.trigger({
+ type: 'shown'
+ , relatedTarget: previous
+ })
+ })
+ }
+
+ , activate: function ( element, container, callback) {
+ var $active = container.find('> .active')
+ , transition = callback
+ && $.support.transition
+ && $active.hasClass('fade')
+
+ function next() {
+ $active
+ .removeClass('active')
+ .find('> .dropdown-menu > .active')
+ .removeClass('active')
+
+ element.addClass('active')
+
+ if (transition) {
+ element[0].offsetWidth // reflow for transition
+ element.addClass('in')
+ } else {
+ element.removeClass('fade')
+ }
+
+ if ( element.parent('.dropdown-menu') ) {
+ element.closest('li.dropdown').addClass('active')
+ }
+
+ callback && callback()
+ }
+
+ transition ?
+ $active.one($.support.transition.end, next) :
+ next()
+
+ $active.removeClass('in')
+ }
+ }
+
+
+ /* TAB PLUGIN DEFINITION
+ * ===================== */
+
+ $.fn.tab = function ( option ) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('tab')
+ if (!data) $this.data('tab', (data = new Tab(this)))
+ if (typeof option == 'string') data[option]()
+ })
+ }
+
+ $.fn.tab.Constructor = Tab
+
+
+ /* TAB DATA-API
+ * ============ */
+
+ $(function () {
+ $('body').on('click.tab.data-api', '[data-toggle="tab"], [data-toggle="pill"]', function (e) {
+ e.preventDefault()
+ $(this).tab('show')
+ })
+ })
+
+}(window.jQuery);/* =============================================================
+ * bootstrap-typeahead.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#typeahead
+ * =============================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ============================================================ */
+
+
+!function($){
+
+ "use strict"; // jshint ;_;
+
+
+ /* TYPEAHEAD PUBLIC CLASS DEFINITION
+ * ================================= */
+
+ var Typeahead = function (element, options) {
+ this.$element = $(element)
+ this.options = $.extend({}, $.fn.typeahead.defaults, options)
+ this.matcher = this.options.matcher || this.matcher
+ this.sorter = this.options.sorter || this.sorter
+ this.highlighter = this.options.highlighter || this.highlighter
+ this.updater = this.options.updater || this.updater
+ this.$menu = $(this.options.menu).appendTo('body')
+ this.source = this.options.source
+ this.shown = false
+ this.listen()
+ }
+
+ Typeahead.prototype = {
+
+ constructor: Typeahead
+
+ , select: function () {
+ var val = this.$menu.find('.active').attr('data-value')
+ this.$element
+ .val(this.updater(val))
+ .change()
+ return this.hide()
+ }
+
+ , updater: function (item) {
+ return item
+ }
+
+ , show: function () {
+ var pos = $.extend({}, this.$element.offset(), {
+ height: this.$element[0].offsetHeight
+ })
+
+ this.$menu.css({
+ top: pos.top + pos.height
+ , left: pos.left
+ })
+
+ this.$menu.show()
+ this.shown = true
+ return this
+ }
+
+ , hide: function () {
+ this.$menu.hide()
+ this.shown = false
+ return this
+ }
+
+ , lookup: function (event) {
+ var items
+
+ this.query = this.$element.val()
+
+ if (!this.query || this.query.length < this.options.minLength) {
+ return this.shown ? this.hide() : this
+ }
+
+ items = $.isFunction(this.source) ? this.source(this.query, $.proxy(this.process, this)) : this.source
+
+ return items ? this.process(items) : this
+ }
+
+ , process: function (items) {
+ var that = this
+
+ items = $.grep(items, function (item) {
+ return that.matcher(item)
+ })
+
+ items = this.sorter(items)
+
+ if (!items.length) {
+ return this.shown ? this.hide() : this
+ }
+
+ return this.render(items.slice(0, this.options.items)).show()
+ }
+
+ , matcher: function (item) {
+ return ~item.toLowerCase().indexOf(this.query.toLowerCase())
+ }
+
+ , sorter: function (items) {
+ var beginswith = []
+ , caseSensitive = []
+ , caseInsensitive = []
+ , item
+
+ while (item = items.shift()) {
+ if (!item.toLowerCase().indexOf(this.query.toLowerCase())) beginswith.push(item)
+ else if (~item.indexOf(this.query)) caseSensitive.push(item)
+ else caseInsensitive.push(item)
+ }
+
+ return beginswith.concat(caseSensitive, caseInsensitive)
+ }
+
+ , highlighter: function (item) {
+ var query = this.query.replace(/[\-\[\]{}()*+?.,\\\^$|#\s]/g, '\\$&')
+ return item.replace(new RegExp('(' + query + ')', 'ig'), function ($1, match) {
+ return '<strong>' + match + '</strong>'
+ })
+ }
+
+ , render: function (items) {
+ var that = this
+
+ items = $(items).map(function (i, item) {
+ i = $(that.options.item).attr('data-value', item)
+ i.find('a').html(that.highlighter(item))
+ return i[0]
+ })
+
+ items.first().addClass('active')
+ this.$menu.html(items)
+ return this
+ }
+
+ , next: function (event) {
+ var active = this.$menu.find('.active').removeClass('active')
+ , next = active.next()
+
+ if (!next.length) {
+ next = $(this.$menu.find('li')[0])
+ }
+
+ next.addClass('active')
+ }
+
+ , prev: function (event) {
+ var active = this.$menu.find('.active').removeClass('active')
+ , prev = active.prev()
+
+ if (!prev.length) {
+ prev = this.$menu.find('li').last()
+ }
+
+ prev.addClass('active')
+ }
+
+ , listen: function () {
+ this.$element
+ .on('blur', $.proxy(this.blur, this))
+ .on('keypress', $.proxy(this.keypress, this))
+ .on('keyup', $.proxy(this.keyup, this))
+
+ if ($.browser.webkit || $.browser.msie) {
+ this.$element.on('keydown', $.proxy(this.keydown, this))
+ }
+
+ this.$menu
+ .on('click', $.proxy(this.click, this))
+ .on('mouseenter', 'li', $.proxy(this.mouseenter, this))
+ }
+
+ , move: function (e) {
+ if (!this.shown) return
+
+ switch(e.keyCode) {
+ case 9: // tab
+ case 13: // enter
+ case 27: // escape
+ e.preventDefault()
+ break
+
+ case 38: // up arrow
+ e.preventDefault()
+ this.prev()
+ break
+
+ case 40: // down arrow
+ e.preventDefault()
+ this.next()
+ break
+ }
+
+ e.stopPropagation()
+ }
+
+ , keydown: function (e) {
+ this.suppressKeyPressRepeat = !~$.inArray(e.keyCode, [40,38,9,13,27])
+ this.move(e)
+ }
+
+ , keypress: function (e) {
+ if (this.suppressKeyPressRepeat) return
+ this.move(e)
+ }
+
+ , keyup: function (e) {
+ switch(e.keyCode) {
+ case 40: // down arrow
+ case 38: // up arrow
+ break
+
+ case 9: // tab
+ case 13: // enter
+ if (!this.shown) return
+ this.select()
+ break
+
+ case 27: // escape
+ if (!this.shown) return
+ this.hide()
+ break
+
+ default:
+ this.lookup()
+ }
+
+ e.stopPropagation()
+ e.preventDefault()
+ }
+
+ , blur: function (e) {
+ var that = this
+ setTimeout(function () { that.hide() }, 150)
+ }
+
+ , click: function (e) {
+ e.stopPropagation()
+ e.preventDefault()
+ this.select()
+ }
+
+ , mouseenter: function (e) {
+ this.$menu.find('.active').removeClass('active')
+ $(e.currentTarget).addClass('active')
+ }
+
+ }
+
+
+ /* TYPEAHEAD PLUGIN DEFINITION
+ * =========================== */
+
+ $.fn.typeahead = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('typeahead')
+ , options = typeof option == 'object' && option
+ if (!data) $this.data('typeahead', (data = new Typeahead(this, options)))
+ if (typeof option == 'string') data[option]()
+ })
+ }
+
+ $.fn.typeahead.defaults = {
+ source: []
+ , items: 8
+ , menu: '<ul class="typeahead dropdown-menu"></ul>'
+ , item: '<li><a href="#"></a></li>'
+ , minLength: 1
+ }
+
+ $.fn.typeahead.Constructor = Typeahead
+
+
+ /* TYPEAHEAD DATA-API
+ * ================== */
+
+ $(function () {
+ $('body').on('focus.typeahead.data-api', '[data-provide="typeahead"]', function (e) {
+ var $this = $(this)
+ if ($this.data('typeahead')) return
+ e.preventDefault()
+ $this.typeahead($this.data())
+ })
+ })
+
+}(window.jQuery);
+/* ==========================================================
+ * bootstrap-affix.js v2.1.0
+ * http://twitter.github.com/bootstrap/javascript.html#affix
+ * ==========================================================
+ * Copyright 2012 Twitter, Inc.
+ *
+ * Licensed 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.
+ * ========================================================== */
+
+
+!function ($) {
+
+ "use strict"; // jshint ;_;
+
+
+ /* AFFIX CLASS DEFINITION
+ * ====================== */
+
+ var Affix = function (element, options) {
+ this.options = $.extend({}, $.fn.affix.defaults, options)
+ this.$window = $(window).on('scroll.affix.data-api', $.proxy(this.checkPosition, this))
+ this.$element = $(element)
+ this.checkPosition()
+ }
+
+ Affix.prototype.checkPosition = function () {
+ if (!this.$element.is(':visible')) return
+
+ var scrollHeight = $(document).height()
+ , scrollTop = this.$window.scrollTop()
+ , position = this.$element.offset()
+ , offset = this.options.offset
+ , offsetBottom = offset.bottom
+ , offsetTop = offset.top
+ , reset = 'affix affix-top affix-bottom'
+ , affix
+
+ if (typeof offset != 'object') offsetBottom = offsetTop = offset
+ if (typeof offsetTop == 'function') offsetTop = offset.top()
+ if (typeof offsetBottom == 'function') offsetBottom = offset.bottom()
+
+ affix = this.unpin != null && (scrollTop + this.unpin <= position.top) ?
+ false : offsetBottom != null && (position.top + this.$element.height() >= scrollHeight - offsetBottom) ?
+ 'bottom' : offsetTop != null && scrollTop <= offsetTop ?
+ 'top' : false
+
+ if (this.affixed === affix) return
+
+ this.affixed = affix
+ this.unpin = affix == 'bottom' ? position.top - scrollTop : null
+
+ this.$element.removeClass(reset).addClass('affix' + (affix ? '-' + affix : ''))
+ }
+
+
+ /* AFFIX PLUGIN DEFINITION
+ * ======================= */
+
+ $.fn.affix = function (option) {
+ return this.each(function () {
+ var $this = $(this)
+ , data = $this.data('affix')
+ , options = typeof option == 'object' && option
+ if (!data) $this.data('affix', (data = new Affix(this, options)))
+ if (typeof option == 'string') data[option]()
+ })
+ }
+
+ $.fn.affix.Constructor = Affix
+
+ $.fn.affix.defaults = {
+ offset: 0
+ }
+
+
+ /* AFFIX DATA-API
+ * ============== */
+
+ $(window).on('load', function () {
+ $('[data-spy="affix"]').each(function () {
+ var $spy = $(this)
+ , data = $spy.data()
+
+ data.offset = data.offset || {}
+
+ data.offsetBottom && (data.offset.bottom = data.offsetBottom)
+ data.offsetTop && (data.offset.top = data.offsetTop)
+
+ $spy.affix(data)
+ })
+ })
+
+
+}(window.jQuery); \ No newline at end of file
diff --git a/docs/js/vendor/bootstrap.min.js b/docs/js/vendor/bootstrap.min.js
new file mode 100755
index 0000000000..66e887bb3f
--- /dev/null
+++ b/docs/js/vendor/bootstrap.min.js
@@ -0,0 +1,6 @@
+/*!
+* Bootstrap.js by @fat & @mdo
+* Copyright 2012 Twitter, Inc.
+* http://www.apache.org/licenses/LICENSE-2.0.txt
+*/
+!function(e){e(function(){"use strict";e.support.transition=function(){var e=function(){var e=document.createElement("bootstrap"),t={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"},n;for(n in t)if(e.style[n]!==undefined)return t[n]}();return e&&{end:e}}()})}(window.jQuery),!function(e){"use strict";var t='[data-dismiss="alert"]',n=function(n){e(n).on("click",t,this.close)};n.prototype.close=function(t){function s(){i.trigger("closed").remove()}var n=e(this),r=n.attr("data-target"),i;r||(r=n.attr("href"),r=r&&r.replace(/.*(?=#[^\s]*$)/,"")),i=e(r),t&&t.preventDefault(),i.length||(i=n.hasClass("alert")?n:n.parent()),i.trigger(t=e.Event("close"));if(t.isDefaultPrevented())return;i.removeClass("in"),e.support.transition&&i.hasClass("fade")?i.on(e.support.transition.end,s):s()},e.fn.alert=function(t){return this.each(function(){var r=e(this),i=r.data("alert");i||r.data("alert",i=new n(this)),typeof t=="string"&&i[t].call(r)})},e.fn.alert.Constructor=n,e(function(){e("body").on("click.alert.data-api",t,n.prototype.close)})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.button.defaults,n)};t.prototype.setState=function(e){var t="disabled",n=this.$element,r=n.data(),i=n.is("input")?"val":"html";e+="Text",r.resetText||n.data("resetText",n[i]()),n[i](r[e]||this.options[e]),setTimeout(function(){e=="loadingText"?n.addClass(t).attr(t,t):n.removeClass(t).removeAttr(t)},0)},t.prototype.toggle=function(){var e=this.$element.parent('[data-toggle="buttons-radio"]');e&&e.find(".active").removeClass("active"),this.$element.toggleClass("active")},e.fn.button=function(n){return this.each(function(){var r=e(this),i=r.data("button"),s=typeof n=="object"&&n;i||r.data("button",i=new t(this,s)),n=="toggle"?i.toggle():n&&i.setState(n)})},e.fn.button.defaults={loadingText:"loading..."},e.fn.button.Constructor=t,e(function(){e("body").on("click.button.data-api","[data-toggle^=button]",function(t){var n=e(t.target);n.hasClass("btn")||(n=n.closest(".btn")),n.button("toggle")})})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=n,this.options.slide&&this.slide(this.options.slide),this.options.pause=="hover"&&this.$element.on("mouseenter",e.proxy(this.pause,this)).on("mouseleave",e.proxy(this.cycle,this))};t.prototype={cycle:function(t){return t||(this.paused=!1),this.options.interval&&!this.paused&&(this.interval=setInterval(e.proxy(this.next,this),this.options.interval)),this},to:function(t){var n=this.$element.find(".item.active"),r=n.parent().children(),i=r.index(n),s=this;if(t>r.length-1||t<0)return;return this.sliding?this.$element.one("slid",function(){s.to(t)}):i==t?this.pause().cycle():this.slide(t>i?"next":"prev",e(r[t]))},pause:function(t){return t||(this.paused=!0),this.$element.find(".next, .prev").length&&e.support.transition.end&&(this.$element.trigger(e.support.transition.end),this.cycle()),clearInterval(this.interval),this.interval=null,this},next:function(){if(this.sliding)return;return this.slide("next")},prev:function(){if(this.sliding)return;return this.slide("prev")},slide:function(t,n){var r=this.$element.find(".item.active"),i=n||r[t](),s=this.interval,o=t=="next"?"left":"right",u=t=="next"?"first":"last",a=this,f=e.Event("slide",{relatedTarget:i[0]});this.sliding=!0,s&&this.pause(),i=i.length?i:this.$element.find(".item")[u]();if(i.hasClass("active"))return;if(e.support.transition&&this.$element.hasClass("slide")){this.$element.trigger(f);if(f.isDefaultPrevented())return;i.addClass(t),i[0].offsetWidth,r.addClass(o),i.addClass(o),this.$element.one(e.support.transition.end,function(){i.removeClass([t,o].join(" ")).addClass("active"),r.removeClass(["active",o].join(" ")),a.sliding=!1,setTimeout(function(){a.$element.trigger("slid")},0)})}else{this.$element.trigger(f);if(f.isDefaultPrevented())return;r.removeClass("active"),i.addClass("active"),this.sliding=!1,this.$element.trigger("slid")}return s&&this.cycle(),this}},e.fn.carousel=function(n){return this.each(function(){var r=e(this),i=r.data("carousel"),s=e.extend({},e.fn.carousel.defaults,typeof n=="object"&&n),o=typeof n=="string"?n:s.slide;i||r.data("carousel",i=new t(this,s)),typeof n=="number"?i.to(n):o?i[o]():s.interval&&i.cycle()})},e.fn.carousel.defaults={interval:5e3,pause:"hover"},e.fn.carousel.Constructor=t,e(function(){e("body").on("click.carousel.data-api","[data-slide]",function(t){var n=e(this),r,i=e(n.attr("data-target")||(r=n.attr("href"))&&r.replace(/.*(?=#[^\s]+$)/,"")),s=!i.data("modal")&&e.extend({},i.data(),n.data());i.carousel(s),t.preventDefault()})})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.collapse.defaults,n),this.options.parent&&(this.$parent=e(this.options.parent)),this.options.toggle&&this.toggle()};t.prototype={constructor:t,dimension:function(){var e=this.$element.hasClass("width");return e?"width":"height"},show:function(){var t,n,r,i;if(this.transitioning)return;t=this.dimension(),n=e.camelCase(["scroll",t].join("-")),r=this.$parent&&this.$parent.find("> .accordion-group > .in");if(r&&r.length){i=r.data("collapse");if(i&&i.transitioning)return;r.collapse("hide"),i||r.data("collapse",null)}this.$element[t](0),this.transition("addClass",e.Event("show"),"shown"),e.support.transition&&this.$element[t](this.$element[0][n])},hide:function(){var t;if(this.transitioning)return;t=this.dimension(),this.reset(this.$element[t]()),this.transition("removeClass",e.Event("hide"),"hidden"),this.$element[t](0)},reset:function(e){var t=this.dimension();return this.$element.removeClass("collapse")[t](e||"auto")[0].offsetWidth,this.$element[e!==null?"addClass":"removeClass"]("collapse"),this},transition:function(t,n,r){var i=this,s=function(){n.type=="show"&&i.reset(),i.transitioning=0,i.$element.trigger(r)};this.$element.trigger(n);if(n.isDefaultPrevented())return;this.transitioning=1,this.$element[t]("in"),e.support.transition&&this.$element.hasClass("collapse")?this.$element.one(e.support.transition.end,s):s()},toggle:function(){this[this.$element.hasClass("in")?"hide":"show"]()}},e.fn.collapse=function(n){return this.each(function(){var r=e(this),i=r.data("collapse"),s=typeof n=="object"&&n;i||r.data("collapse",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.collapse.defaults={toggle:!0},e.fn.collapse.Constructor=t,e(function(){e("body").on("click.collapse.data-api","[data-toggle=collapse]",function(t){var n=e(this),r,i=n.attr("data-target")||t.preventDefault()||(r=n.attr("href"))&&r.replace(/.*(?=#[^\s]+$)/,""),s=e(i).data("collapse")?"toggle":n.data();n[e(i).hasClass("in")?"addClass":"removeClass"]("collapsed"),e(i).collapse(s)})})}(window.jQuery),!function(e){"use strict";function r(){i(e(t)).removeClass("open")}function i(t){var n=t.attr("data-target"),r;return n||(n=t.attr("href"),n=n&&n.replace(/.*(?=#[^\s]*$)/,"")),r=e(n),r.length||(r=t.parent()),r}var t="[data-toggle=dropdown]",n=function(t){var n=e(t).on("click.dropdown.data-api",this.toggle);e("html").on("click.dropdown.data-api",function(){n.parent().removeClass("open")})};n.prototype={constructor:n,toggle:function(t){var n=e(this),s,o;if(n.is(".disabled, :disabled"))return;return s=i(n),o=s.hasClass("open"),r(),o||(s.toggleClass("open"),n.focus()),!1},keydown:function(t){var n,r,s,o,u,a;if(!/(38|40|27)/.test(t.keyCode))return;n=e(this),t.preventDefault(),t.stopPropagation();if(n.is(".disabled, :disabled"))return;o=i(n),u=o.hasClass("open");if(!u||u&&t.keyCode==27)return n.click();r=e("[role=menu] li:not(.divider) a",o);if(!r.length)return;a=r.index(r.filter(":focus")),t.keyCode==38&&a>0&&a--,t.keyCode==40&&a<r.length-1&&a++,~a||(a=0),r.eq(a).focus()}},e.fn.dropdown=function(t){return this.each(function(){var r=e(this),i=r.data("dropdown");i||r.data("dropdown",i=new n(this)),typeof t=="string"&&i[t].call(r)})},e.fn.dropdown.Constructor=n,e(function(){e("html").on("click.dropdown.data-api touchstart.dropdown.data-api",r),e("body").on("click.dropdown touchstart.dropdown.data-api",".dropdown",function(e){e.stopPropagation()}).on("click.dropdown.data-api touchstart.dropdown.data-api",t,n.prototype.toggle).on("keydown.dropdown.data-api touchstart.dropdown.data-api",t+", [role=menu]",n.prototype.keydown)})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.options=n,this.$element=e(t).delegate('[data-dismiss="modal"]',"click.dismiss.modal",e.proxy(this.hide,this)),this.options.remote&&this.$element.find(".modal-body").load(this.options.remote)};t.prototype={constructor:t,toggle:function(){return this[this.isShown?"hide":"show"]()},show:function(){var t=this,n=e.Event("show");this.$element.trigger(n);if(this.isShown||n.isDefaultPrevented())return;e("body").addClass("modal-open"),this.isShown=!0,this.escape(),this.backdrop(function(){var n=e.support.transition&&t.$element.hasClass("fade");t.$element.parent().length||t.$element.appendTo(document.body),t.$element.show(),n&&t.$element[0].offsetWidth,t.$element.addClass("in").attr("aria-hidden",!1).focus(),t.enforceFocus(),n?t.$element.one(e.support.transition.end,function(){t.$element.trigger("shown")}):t.$element.trigger("shown")})},hide:function(t){t&&t.preventDefault();var n=this;t=e.Event("hide"),this.$element.trigger(t);if(!this.isShown||t.isDefaultPrevented())return;this.isShown=!1,e("body").removeClass("modal-open"),this.escape(),e(document).off("focusin.modal"),this.$element.removeClass("in").attr("aria-hidden",!0),e.support.transition&&this.$element.hasClass("fade")?this.hideWithTransition():this.hideModal()},enforceFocus:function(){var t=this;e(document).on("focusin.modal",function(e){t.$element[0]!==e.target&&!t.$element.has(e.target).length&&t.$element.focus()})},escape:function(){var e=this;this.isShown&&this.options.keyboard?this.$element.on("keyup.dismiss.modal",function(t){t.which==27&&e.hide()}):this.isShown||this.$element.off("keyup.dismiss.modal")},hideWithTransition:function(){var t=this,n=setTimeout(function(){t.$element.off(e.support.transition.end),t.hideModal()},500);this.$element.one(e.support.transition.end,function(){clearTimeout(n),t.hideModal()})},hideModal:function(e){this.$element.hide().trigger("hidden"),this.backdrop()},removeBackdrop:function(){this.$backdrop.remove(),this.$backdrop=null},backdrop:function(t){var n=this,r=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var i=e.support.transition&&r;this.$backdrop=e('<div class="modal-backdrop '+r+'" />').appendTo(document.body),this.options.backdrop!="static"&&this.$backdrop.click(e.proxy(this.hide,this)),i&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),i?this.$backdrop.one(e.support.transition.end,t):t()}else!this.isShown&&this.$backdrop?(this.$backdrop.removeClass("in"),e.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one(e.support.transition.end,e.proxy(this.removeBackdrop,this)):this.removeBackdrop()):t&&t()}},e.fn.modal=function(n){return this.each(function(){var r=e(this),i=r.data("modal"),s=e.extend({},e.fn.modal.defaults,r.data(),typeof n=="object"&&n);i||r.data("modal",i=new t(this,s)),typeof n=="string"?i[n]():s.show&&i.show()})},e.fn.modal.defaults={backdrop:!0,keyboard:!0,show:!0},e.fn.modal.Constructor=t,e(function(){e("body").on("click.modal.data-api",'[data-toggle="modal"]',function(t){var n=e(this),r=n.attr("href"),i=e(n.attr("data-target")||r&&r.replace(/.*(?=#[^\s]+$)/,"")),s=i.data("modal")?"toggle":e.extend({remote:!/#/.test(r)&&r},i.data(),n.data());t.preventDefault(),i.modal(s).one("hide",function(){n.focus()})})})}(window.jQuery),!function(e){"use strict";var t=function(e,t){this.init("tooltip",e,t)};t.prototype={constructor:t,init:function(t,n,r){var i,s;this.type=t,this.$element=e(n),this.options=this.getOptions(r),this.enabled=!0,this.options.trigger=="click"?this.$element.on("click."+this.type,this.options.selector,e.proxy(this.toggle,this)):this.options.trigger!="manual"&&(i=this.options.trigger=="hover"?"mouseenter":"focus",s=this.options.trigger=="hover"?"mouseleave":"blur",this.$element.on(i+"."+this.type,this.options.selector,e.proxy(this.enter,this)),this.$element.on(s+"."+this.type,this.options.selector,e.proxy(this.leave,this))),this.options.selector?this._options=e.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},getOptions:function(t){return t=e.extend({},e.fn[this.type].defaults,t,this.$element.data()),t.delay&&typeof t.delay=="number"&&(t.delay={show:t.delay,hide:t.delay}),t},enter:function(t){var n=e(t.currentTarget)[this.type](this._options).data(this.type);if(!n.options.delay||!n.options.delay.show)return n.show();clearTimeout(this.timeout),n.hoverState="in",this.timeout=setTimeout(function(){n.hoverState=="in"&&n.show()},n.options.delay.show)},leave:function(t){var n=e(t.currentTarget)[this.type](this._options).data(this.type);this.timeout&&clearTimeout(this.timeout);if(!n.options.delay||!n.options.delay.hide)return n.hide();n.hoverState="out",this.timeout=setTimeout(function(){n.hoverState=="out"&&n.hide()},n.options.delay.hide)},show:function(){var e,t,n,r,i,s,o;if(this.hasContent()&&this.enabled){e=this.tip(),this.setContent(),this.options.animation&&e.addClass("fade"),s=typeof this.options.placement=="function"?this.options.placement.call(this,e[0],this.$element[0]):this.options.placement,t=/in/.test(s),e.remove().css({top:0,left:0,display:"block"}).appendTo(t?this.$element:document.body),n=this.getPosition(t),r=e[0].offsetWidth,i=e[0].offsetHeight;switch(t?s.split(" ")[1]:s){case"bottom":o={top:n.top+n.height,left:n.left+n.width/2-r/2};break;case"top":o={top:n.top-i,left:n.left+n.width/2-r/2};break;case"left":o={top:n.top+n.height/2-i/2,left:n.left-r};break;case"right":o={top:n.top+n.height/2-i/2,left:n.left+n.width}}e.css(o).addClass(s).addClass("in")}},setContent:function(){var e=this.tip(),t=this.getTitle();e.find(".tooltip-inner")[this.options.html?"html":"text"](t),e.removeClass("fade in top bottom left right")},hide:function(){function r(){var t=setTimeout(function(){n.off(e.support.transition.end).remove()},500);n.one(e.support.transition.end,function(){clearTimeout(t),n.remove()})}var t=this,n=this.tip();return n.removeClass("in"),e.support.transition&&this.$tip.hasClass("fade")?r():n.remove(),this},fixTitle:function(){var e=this.$element;(e.attr("title")||typeof e.attr("data-original-title")!="string")&&e.attr("data-original-title",e.attr("title")||"").removeAttr("title")},hasContent:function(){return this.getTitle()},getPosition:function(t){return e.extend({},t?{top:0,left:0}:this.$element.offset(),{width:this.$element[0].offsetWidth,height:this.$element[0].offsetHeight})},getTitle:function(){var e,t=this.$element,n=this.options;return e=t.attr("data-original-title")||(typeof n.title=="function"?n.title.call(t[0]):n.title),e},tip:function(){return this.$tip=this.$tip||e(this.options.template)},validate:function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},enable:function(){this.enabled=!0},disable:function(){this.enabled=!1},toggleEnabled:function(){this.enabled=!this.enabled},toggle:function(){this[this.tip().hasClass("in")?"hide":"show"]()},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}},e.fn.tooltip=function(n){return this.each(function(){var r=e(this),i=r.data("tooltip"),s=typeof n=="object"&&n;i||r.data("tooltip",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.tooltip.Constructor=t,e.fn.tooltip.defaults={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover",title:"",delay:0,html:!0}}(window.jQuery),!function(e){"use strict";var t=function(e,t){this.init("popover",e,t)};t.prototype=e.extend({},e.fn.tooltip.Constructor.prototype,{constructor:t,setContent:function(){var e=this.tip(),t=this.getTitle(),n=this.getContent();e.find(".popover-title")[this.options.html?"html":"text"](t),e.find(".popover-content > *")[this.options.html?"html":"text"](n),e.removeClass("fade top bottom left right in")},hasContent:function(){return this.getTitle()||this.getContent()},getContent:function(){var e,t=this.$element,n=this.options;return e=t.attr("data-content")||(typeof n.content=="function"?n.content.call(t[0]):n.content),e},tip:function(){return this.$tip||(this.$tip=e(this.options.template)),this.$tip},destroy:function(){this.hide().$element.off("."+this.type).removeData(this.type)}}),e.fn.popover=function(n){return this.each(function(){var r=e(this),i=r.data("popover"),s=typeof n=="object"&&n;i||r.data("popover",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.popover.Constructor=t,e.fn.popover.defaults=e.extend({},e.fn.tooltip.defaults,{placement:"right",trigger:"click",content:"",template:'<div class="popover"><div class="arrow"></div><div class="popover-inner"><h3 class="popover-title"></h3><div class="popover-content"><p></p></div></div></div>'})}(window.jQuery),!function(e){"use strict";function t(t,n){var r=e.proxy(this.process,this),i=e(t).is("body")?e(window):e(t),s;this.options=e.extend({},e.fn.scrollspy.defaults,n),this.$scrollElement=i.on("scroll.scroll-spy.data-api",r),this.selector=(this.options.target||(s=e(t).attr("href"))&&s.replace(/.*(?=#[^\s]+$)/,"")||"")+" .nav li > a",this.$body=e("body"),this.refresh(),this.process()}t.prototype={constructor:t,refresh:function(){var t=this,n;this.offsets=e([]),this.targets=e([]),n=this.$body.find(this.selector).map(function(){var t=e(this),n=t.data("target")||t.attr("href"),r=/^#\w/.test(n)&&e(n);return r&&r.length&&[[r.position().top,n]]||null}).sort(function(e,t){return e[0]-t[0]}).each(function(){t.offsets.push(this[0]),t.targets.push(this[1])})},process:function(){var e=this.$scrollElement.scrollTop()+this.options.offset,t=this.$scrollElement[0].scrollHeight||this.$body[0].scrollHeight,n=t-this.$scrollElement.height(),r=this.offsets,i=this.targets,s=this.activeTarget,o;if(e>=n)return s!=(o=i.last()[0])&&this.activate(o);for(o=r.length;o--;)s!=i[o]&&e>=r[o]&&(!r[o+1]||e<=r[o+1])&&this.activate(i[o])},activate:function(t){var n,r;this.activeTarget=t,e(this.selector).parent(".active").removeClass("active"),r=this.selector+'[data-target="'+t+'"],'+this.selector+'[href="'+t+'"]',n=e(r).parent("li").addClass("active"),n.parent(".dropdown-menu").length&&(n=n.closest("li.dropdown").addClass("active")),n.trigger("activate")}},e.fn.scrollspy=function(n){return this.each(function(){var r=e(this),i=r.data("scrollspy"),s=typeof n=="object"&&n;i||r.data("scrollspy",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.scrollspy.Constructor=t,e.fn.scrollspy.defaults={offset:10},e(window).on("load",function(){e('[data-spy="scroll"]').each(function(){var t=e(this);t.scrollspy(t.data())})})}(window.jQuery),!function(e){"use strict";var t=function(t){this.element=e(t)};t.prototype={constructor:t,show:function(){var t=this.element,n=t.closest("ul:not(.dropdown-menu)"),r=t.attr("data-target"),i,s,o;r||(r=t.attr("href"),r=r&&r.replace(/.*(?=#[^\s]*$)/,""));if(t.parent("li").hasClass("active"))return;i=n.find(".active a").last()[0],o=e.Event("show",{relatedTarget:i}),t.trigger(o);if(o.isDefaultPrevented())return;s=e(r),this.activate(t.parent("li"),n),this.activate(s,s.parent(),function(){t.trigger({type:"shown",relatedTarget:i})})},activate:function(t,n,r){function o(){i.removeClass("active").find("> .dropdown-menu > .active").removeClass("active"),t.addClass("active"),s?(t[0].offsetWidth,t.addClass("in")):t.removeClass("fade"),t.parent(".dropdown-menu")&&t.closest("li.dropdown").addClass("active"),r&&r()}var i=n.find("> .active"),s=r&&e.support.transition&&i.hasClass("fade");s?i.one(e.support.transition.end,o):o(),i.removeClass("in")}},e.fn.tab=function(n){return this.each(function(){var r=e(this),i=r.data("tab");i||r.data("tab",i=new t(this)),typeof n=="string"&&i[n]()})},e.fn.tab.Constructor=t,e(function(){e("body").on("click.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(t){t.preventDefault(),e(this).tab("show")})})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.$element=e(t),this.options=e.extend({},e.fn.typeahead.defaults,n),this.matcher=this.options.matcher||this.matcher,this.sorter=this.options.sorter||this.sorter,this.highlighter=this.options.highlighter||this.highlighter,this.updater=this.options.updater||this.updater,this.$menu=e(this.options.menu).appendTo("body"),this.source=this.options.source,this.shown=!1,this.listen()};t.prototype={constructor:t,select:function(){var e=this.$menu.find(".active").attr("data-value");return this.$element.val(this.updater(e)).change(),this.hide()},updater:function(e){return e},show:function(){var t=e.extend({},this.$element.offset(),{height:this.$element[0].offsetHeight});return this.$menu.css({top:t.top+t.height,left:t.left}),this.$menu.show(),this.shown=!0,this},hide:function(){return this.$menu.hide(),this.shown=!1,this},lookup:function(t){var n;return this.query=this.$element.val(),!this.query||this.query.length<this.options.minLength?this.shown?this.hide():this:(n=e.isFunction(this.source)?this.source(this.query,e.proxy(this.process,this)):this.source,n?this.process(n):this)},process:function(t){var n=this;return t=e.grep(t,function(e){return n.matcher(e)}),t=this.sorter(t),t.length?this.render(t.slice(0,this.options.items)).show():this.shown?this.hide():this},matcher:function(e){return~e.toLowerCase().indexOf(this.query.toLowerCase())},sorter:function(e){var t=[],n=[],r=[],i;while(i=e.shift())i.toLowerCase().indexOf(this.query.toLowerCase())?~i.indexOf(this.query)?n.push(i):r.push(i):t.push(i);return t.concat(n,r)},highlighter:function(e){var t=this.query.replace(/[\-\[\]{}()*+?.,\\\^$|#\s]/g,"\\$&");return e.replace(new RegExp("("+t+")","ig"),function(e,t){return"<strong>"+t+"</strong>"})},render:function(t){var n=this;return t=e(t).map(function(t,r){return t=e(n.options.item).attr("data-value",r),t.find("a").html(n.highlighter(r)),t[0]}),t.first().addClass("active"),this.$menu.html(t),this},next:function(t){var n=this.$menu.find(".active").removeClass("active"),r=n.next();r.length||(r=e(this.$menu.find("li")[0])),r.addClass("active")},prev:function(e){var t=this.$menu.find(".active").removeClass("active"),n=t.prev();n.length||(n=this.$menu.find("li").last()),n.addClass("active")},listen:function(){this.$element.on("blur",e.proxy(this.blur,this)).on("keypress",e.proxy(this.keypress,this)).on("keyup",e.proxy(this.keyup,this)),(e.browser.webkit||e.browser.msie)&&this.$element.on("keydown",e.proxy(this.keydown,this)),this.$menu.on("click",e.proxy(this.click,this)).on("mouseenter","li",e.proxy(this.mouseenter,this))},move:function(e){if(!this.shown)return;switch(e.keyCode){case 9:case 13:case 27:e.preventDefault();break;case 38:e.preventDefault(),this.prev();break;case 40:e.preventDefault(),this.next()}e.stopPropagation()},keydown:function(t){this.suppressKeyPressRepeat=!~e.inArray(t.keyCode,[40,38,9,13,27]),this.move(t)},keypress:function(e){if(this.suppressKeyPressRepeat)return;this.move(e)},keyup:function(e){switch(e.keyCode){case 40:case 38:break;case 9:case 13:if(!this.shown)return;this.select();break;case 27:if(!this.shown)return;this.hide();break;default:this.lookup()}e.stopPropagation(),e.preventDefault()},blur:function(e){var t=this;setTimeout(function(){t.hide()},150)},click:function(e){e.stopPropagation(),e.preventDefault(),this.select()},mouseenter:function(t){this.$menu.find(".active").removeClass("active"),e(t.currentTarget).addClass("active")}},e.fn.typeahead=function(n){return this.each(function(){var r=e(this),i=r.data("typeahead"),s=typeof n=="object"&&n;i||r.data("typeahead",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.typeahead.defaults={source:[],items:8,menu:'<ul class="typeahead dropdown-menu"></ul>',item:'<li><a href="#"></a></li>',minLength:1},e.fn.typeahead.Constructor=t,e(function(){e("body").on("focus.typeahead.data-api",'[data-provide="typeahead"]',function(t){var n=e(this);if(n.data("typeahead"))return;t.preventDefault(),n.typeahead(n.data())})})}(window.jQuery),!function(e){"use strict";var t=function(t,n){this.options=e.extend({},e.fn.affix.defaults,n),this.$window=e(window).on("scroll.affix.data-api",e.proxy(this.checkPosition,this)),this.$element=e(t),this.checkPosition()};t.prototype.checkPosition=function(){if(!this.$element.is(":visible"))return;var t=e(document).height(),n=this.$window.scrollTop(),r=this.$element.offset(),i=this.options.offset,s=i.bottom,o=i.top,u="affix affix-top affix-bottom",a;typeof i!="object"&&(s=o=i),typeof o=="function"&&(o=i.top()),typeof s=="function"&&(s=i.bottom()),a=this.unpin!=null&&n+this.unpin<=r.top?!1:s!=null&&r.top+this.$element.height()>=t-s?"bottom":o!=null&&n<=o?"top":!1;if(this.affixed===a)return;this.affixed=a,this.unpin=a=="bottom"?r.top-n:null,this.$element.removeClass(u).addClass("affix"+(a?"-"+a:""))},e.fn.affix=function(n){return this.each(function(){var r=e(this),i=r.data("affix"),s=typeof n=="object"&&n;i||r.data("affix",i=new t(this,s)),typeof n=="string"&&i[n]()})},e.fn.affix.Constructor=t,e.fn.affix.defaults={offset:0},e(window).on("load",function(){e('[data-spy="affix"]').each(function(){var t=e(this),n=t.data();n.offset=n.offset||{},n.offsetBottom&&(n.offset.bottom=n.offsetBottom),n.offsetTop&&(n.offset.top=n.offsetTop),t.affix(n)})})}(window.jQuery); \ No newline at end of file
diff --git a/docs/js/vendor/jquery-1.8.0.min.js b/docs/js/vendor/jquery-1.8.0.min.js
new file mode 100755
index 0000000000..066d72c7e3
--- /dev/null
+++ b/docs/js/vendor/jquery-1.8.0.min.js
@@ -0,0 +1,2 @@
+/*! jQuery v@1.8.0 jquery.com | jquery.org/license */
+(function(a,b){function G(a){var b=F[a]={};return p.each(a.split(s),function(a,c){b[c]=!0}),b}function J(a,c,d){if(d===b&&a.nodeType===1){var e="data-"+c.replace(I,"-$1").toLowerCase();d=a.getAttribute(e);if(typeof d=="string"){try{d=d==="true"?!0:d==="false"?!1:d==="null"?null:+d+""===d?+d:H.test(d)?p.parseJSON(d):d}catch(f){}p.data(a,c,d)}else d=b}return d}function K(a){var b;for(b in a){if(b==="data"&&p.isEmptyObject(a[b]))continue;if(b!=="toJSON")return!1}return!0}function ba(){return!1}function bb(){return!0}function bh(a){return!a||!a.parentNode||a.parentNode.nodeType===11}function bi(a,b){do a=a[b];while(a&&a.nodeType!==1);return a}function bj(a,b,c){b=b||0;if(p.isFunction(b))return p.grep(a,function(a,d){var e=!!b.call(a,d,a);return e===c});if(b.nodeType)return p.grep(a,function(a,d){return a===b===c});if(typeof b=="string"){var d=p.grep(a,function(a){return a.nodeType===1});if(be.test(b))return p.filter(b,d,!c);b=p.filter(b,d)}return p.grep(a,function(a,d){return p.inArray(a,b)>=0===c})}function bk(a){var b=bl.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}function bC(a,b){return a.getElementsByTagName(b)[0]||a.appendChild(a.ownerDocument.createElement(b))}function bD(a,b){if(b.nodeType!==1||!p.hasData(a))return;var c,d,e,f=p._data(a),g=p._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;d<e;d++)p.event.add(b,c,h[c][d])}g.data&&(g.data=p.extend({},g.data))}function bE(a,b){var c;if(b.nodeType!==1)return;b.clearAttributes&&b.clearAttributes(),b.mergeAttributes&&b.mergeAttributes(a),c=b.nodeName.toLowerCase(),c==="object"?(b.parentNode&&(b.outerHTML=a.outerHTML),p.support.html5Clone&&a.innerHTML&&!p.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):c==="input"&&bv.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):c==="option"?b.selected=a.defaultSelected:c==="input"||c==="textarea"?b.defaultValue=a.defaultValue:c==="script"&&b.text!==a.text&&(b.text=a.text),b.removeAttribute(p.expando)}function bF(a){return typeof a.getElementsByTagName!="undefined"?a.getElementsByTagName("*"):typeof a.querySelectorAll!="undefined"?a.querySelectorAll("*"):[]}function bG(a){bv.test(a.type)&&(a.defaultChecked=a.checked)}function bX(a,b){if(b in a)return b;var c=b.charAt(0).toUpperCase()+b.slice(1),d=b,e=bV.length;while(e--){b=bV[e]+c;if(b in a)return b}return d}function bY(a,b){return a=b||a,p.css(a,"display")==="none"||!p.contains(a.ownerDocument,a)}function bZ(a,b){var c,d,e=[],f=0,g=a.length;for(;f<g;f++){c=a[f];if(!c.style)continue;e[f]=p._data(c,"olddisplay"),b?(!e[f]&&c.style.display==="none"&&(c.style.display=""),c.style.display===""&&bY(c)&&(e[f]=p._data(c,"olddisplay",cb(c.nodeName)))):(d=bH(c,"display"),!e[f]&&d!=="none"&&p._data(c,"olddisplay",d))}for(f=0;f<g;f++){c=a[f];if(!c.style)continue;if(!b||c.style.display==="none"||c.style.display==="")c.style.display=b?e[f]||"":"none"}return a}function b$(a,b,c){var d=bO.exec(b);return d?Math.max(0,d[1]-(c||0))+(d[2]||"px"):b}function b_(a,b,c,d){var e=c===(d?"border":"content")?4:b==="width"?1:0,f=0;for(;e<4;e+=2)c==="margin"&&(f+=p.css(a,c+bU[e],!0)),d?(c==="content"&&(f-=parseFloat(bH(a,"padding"+bU[e]))||0),c!=="margin"&&(f-=parseFloat(bH(a,"border"+bU[e]+"Width"))||0)):(f+=parseFloat(bH(a,"padding"+bU[e]))||0,c!=="padding"&&(f+=parseFloat(bH(a,"border"+bU[e]+"Width"))||0));return f}function ca(a,b,c){var d=b==="width"?a.offsetWidth:a.offsetHeight,e=!0,f=p.support.boxSizing&&p.css(a,"boxSizing")==="border-box";if(d<=0){d=bH(a,b);if(d<0||d==null)d=a.style[b];if(bP.test(d))return d;e=f&&(p.support.boxSizingReliable||d===a.style[b]),d=parseFloat(d)||0}return d+b_(a,b,c||(f?"border":"content"),e)+"px"}function cb(a){if(bR[a])return bR[a];var b=p("<"+a+">").appendTo(e.body),c=b.css("display");b.remove();if(c==="none"||c===""){bI=e.body.appendChild(bI||p.extend(e.createElement("iframe"),{frameBorder:0,width:0,height:0}));if(!bJ||!bI.createElement)bJ=(bI.contentWindow||bI.contentDocument).document,bJ.write("<!doctype html><html><body>"),bJ.close();b=bJ.body.appendChild(bJ.createElement(a)),c=bH(b,"display"),e.body.removeChild(bI)}return bR[a]=c,c}function ch(a,b,c,d){var e;if(p.isArray(b))p.each(b,function(b,e){c||cd.test(a)?d(a,e):ch(a+"["+(typeof e=="object"?b:"")+"]",e,c,d)});else if(!c&&p.type(b)==="object")for(e in b)ch(a+"["+e+"]",b[e],c,d);else d(a,b)}function cy(a){return function(b,c){typeof b!="string"&&(c=b,b="*");var d,e,f,g=b.toLowerCase().split(s),h=0,i=g.length;if(p.isFunction(c))for(;h<i;h++)d=g[h],f=/^\+/.test(d),f&&(d=d.substr(1)||"*"),e=a[d]=a[d]||[],e[f?"unshift":"push"](c)}}function cz(a,c,d,e,f,g){f=f||c.dataTypes[0],g=g||{},g[f]=!0;var h,i=a[f],j=0,k=i?i.length:0,l=a===cu;for(;j<k&&(l||!h);j++)h=i[j](c,d,e),typeof h=="string"&&(!l||g[h]?h=b:(c.dataTypes.unshift(h),h=cz(a,c,d,e,h,g)));return(l||!h)&&!g["*"]&&(h=cz(a,c,d,e,"*",g)),h}function cA(a,c){var d,e,f=p.ajaxSettings.flatOptions||{};for(d in c)c[d]!==b&&((f[d]?a:e||(e={}))[d]=c[d]);e&&p.extend(!0,a,e)}function cB(a,c,d){var e,f,g,h,i=a.contents,j=a.dataTypes,k=a.responseFields;for(f in k)f in d&&(c[k[f]]=d[f]);while(j[0]==="*")j.shift(),e===b&&(e=a.mimeType||c.getResponseHeader("content-type"));if(e)for(f in i)if(i[f]&&i[f].test(e)){j.unshift(f);break}if(j[0]in d)g=j[0];else{for(f in d){if(!j[0]||a.converters[f+" "+j[0]]){g=f;break}h||(h=f)}g=g||h}if(g)return g!==j[0]&&j.unshift(g),d[g]}function cC(a,b){var c,d,e,f,g=a.dataTypes.slice(),h=g[0],i={},j=0;a.dataFilter&&(b=a.dataFilter(b,a.dataType));if(g[1])for(c in a.converters)i[c.toLowerCase()]=a.converters[c];for(;e=g[++j];)if(e!=="*"){if(h!=="*"&&h!==e){c=i[h+" "+e]||i["* "+e];if(!c)for(d in i){f=d.split(" ");if(f[1]===e){c=i[h+" "+f[0]]||i["* "+f[0]];if(c){c===!0?c=i[d]:i[d]!==!0&&(e=f[0],g.splice(j--,0,e));break}}}if(c!==!0)if(c&&a["throws"])b=c(b);else try{b=c(b)}catch(k){return{state:"parsererror",error:c?k:"No conversion from "+h+" to "+e}}}h=e}return{state:"success",data:b}}function cK(){try{return new a.XMLHttpRequest}catch(b){}}function cL(){try{return new a.ActiveXObject("Microsoft.XMLHTTP")}catch(b){}}function cT(){return setTimeout(function(){cM=b},0),cM=p.now()}function cU(a,b){p.each(b,function(b,c){var d=(cS[b]||[]).concat(cS["*"]),e=0,f=d.length;for(;e<f;e++)if(d[e].call(a,b,c))return})}function cV(a,b,c){var d,e=0,f=0,g=cR.length,h=p.Deferred().always(function(){delete i.elem}),i=function(){var b=cM||cT(),c=Math.max(0,j.startTime+j.duration-b),d=1-(c/j.duration||0),e=0,f=j.tweens.length;for(;e<f;e++)j.tweens[e].run(d);return h.notifyWith(a,[j,d,c]),d<1&&f?c:(h.resolveWith(a,[j]),!1)},j=h.promise({elem:a,props:p.extend({},b),opts:p.extend(!0,{specialEasing:{}},c),originalProperties:b,originalOptions:c,startTime:cM||cT(),duration:c.duration,tweens:[],createTween:function(b,c,d){var e=p.Tween(a,j.opts,b,c,j.opts.specialEasing[b]||j.opts.easing);return j.tweens.push(e),e},stop:function(b){var c=0,d=b?j.tweens.length:0;for(;c<d;c++)j.tweens[c].run(1);return b?h.resolveWith(a,[j,b]):h.rejectWith(a,[j,b]),this}}),k=j.props;cW(k,j.opts.specialEasing);for(;e<g;e++){d=cR[e].call(j,a,k,j.opts);if(d)return d}return cU(j,k),p.isFunction(j.opts.start)&&j.opts.start.call(a,j),p.fx.timer(p.extend(i,{anim:j,queue:j.opts.queue,elem:a})),j.progress(j.opts.progress).done(j.opts.done,j.opts.complete).fail(j.opts.fail).always(j.opts.always)}function cW(a,b){var c,d,e,f,g;for(c in a){d=p.camelCase(c),e=b[d],f=a[c],p.isArray(f)&&(e=f[1],f=a[c]=f[0]),c!==d&&(a[d]=f,delete a[c]),g=p.cssHooks[d];if(g&&"expand"in g){f=g.expand(f),delete a[d];for(c in f)c in a||(a[c]=f[c],b[c]=e)}else b[d]=e}}function cX(a,b,c){var d,e,f,g,h,i,j,k,l=this,m=a.style,n={},o=[],q=a.nodeType&&bY(a);c.queue||(j=p._queueHooks(a,"fx"),j.unqueued==null&&(j.unqueued=0,k=j.empty.fire,j.empty.fire=function(){j.unqueued||k()}),j.unqueued++,l.always(function(){l.always(function(){j.unqueued--,p.queue(a,"fx").length||j.empty.fire()})})),a.nodeType===1&&("height"in b||"width"in b)&&(c.overflow=[m.overflow,m.overflowX,m.overflowY],p.css(a,"display")==="inline"&&p.css(a,"float")==="none"&&(!p.support.inlineBlockNeedsLayout||cb(a.nodeName)==="inline"?m.display="inline-block":m.zoom=1)),c.overflow&&(m.overflow="hidden",p.support.shrinkWrapBlocks||l.done(function(){m.overflow=c.overflow[0],m.overflowX=c.overflow[1],m.overflowY=c.overflow[2]}));for(d in b){f=b[d];if(cO.exec(f)){delete b[d];if(f===(q?"hide":"show"))continue;o.push(d)}}g=o.length;if(g){h=p._data(a,"fxshow")||p._data(a,"fxshow",{}),q?p(a).show():l.done(function(){p(a).hide()}),l.done(function(){var b;p.removeData(a,"fxshow",!0);for(b in n)p.style(a,b,n[b])});for(d=0;d<g;d++)e=o[d],i=l.createTween(e,q?h[e]:0),n[e]=h[e]||p.style(a,e),e in h||(h[e]=i.start,q&&(i.end=i.start,i.start=e==="width"||e==="height"?1:0))}}function cY(a,b,c,d,e){return new cY.prototype.init(a,b,c,d,e)}function cZ(a,b){var c,d={height:a},e=0;for(;e<4;e+=2-b)c=bU[e],d["margin"+c]=d["padding"+c]=a;return b&&(d.opacity=d.width=a),d}function c_(a){return p.isWindow(a)?a:a.nodeType===9?a.defaultView||a.parentWindow:!1}var c,d,e=a.document,f=a.location,g=a.navigator,h=a.jQuery,i=a.$,j=Array.prototype.push,k=Array.prototype.slice,l=Array.prototype.indexOf,m=Object.prototype.toString,n=Object.prototype.hasOwnProperty,o=String.prototype.trim,p=function(a,b){return new p.fn.init(a,b,c)},q=/[\-+]?(?:\d*\.|)\d+(?:[eE][\-+]?\d+|)/.source,r=/\S/,s=/\s+/,t=r.test(" ")?/^[\s\xA0]+|[\s\xA0]+$/g:/^\s+|\s+$/g,u=/^(?:[^#<]*(<[\w\W]+>)[^>]*$|#([\w\-]*)$)/,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^[\],:{}\s]*$/,x=/(?:^|:|,)(?:\s*\[)+/g,y=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,z=/"[^"\\\r\n]*"|true|false|null|-?(?:\d\d*\.|)\d+(?:[eE][\-+]?\d+|)/g,A=/^-ms-/,B=/-([\da-z])/gi,C=function(a,b){return(b+"").toUpperCase()},D=function(){e.addEventListener?(e.removeEventListener("DOMContentLoaded",D,!1),p.ready()):e.readyState==="complete"&&(e.detachEvent("onreadystatechange",D),p.ready())},E={};p.fn=p.prototype={constructor:p,init:function(a,c,d){var f,g,h,i;if(!a)return this;if(a.nodeType)return this.context=this[0]=a,this.length=1,this;if(typeof a=="string"){a.charAt(0)==="<"&&a.charAt(a.length-1)===">"&&a.length>=3?f=[null,a,null]:f=u.exec(a);if(f&&(f[1]||!c)){if(f[1])return c=c instanceof p?c[0]:c,i=c&&c.nodeType?c.ownerDocument||c:e,a=p.parseHTML(f[1],i,!0),v.test(f[1])&&p.isPlainObject(c)&&this.attr.call(a,c,!0),p.merge(this,a);g=e.getElementById(f[2]);if(g&&g.parentNode){if(g.id!==f[2])return d.find(a);this.length=1,this[0]=g}return this.context=e,this.selector=a,this}return!c||c.jquery?(c||d).find(a):this.constructor(c).find(a)}return p.isFunction(a)?d.ready(a):(a.selector!==b&&(this.selector=a.selector,this.context=a.context),p.makeArray(a,this))},selector:"",jquery:"1.8.0",length:0,size:function(){return this.length},toArray:function(){return k.call(this)},get:function(a){return a==null?this.toArray():a<0?this[this.length+a]:this[a]},pushStack:function(a,b,c){var d=p.merge(this.constructor(),a);return d.prevObject=this,d.context=this.context,b==="find"?d.selector=this.selector+(this.selector?" ":"")+c:b&&(d.selector=this.selector+"."+b+"("+c+")"),d},each:function(a,b){return p.each(this,a,b)},ready:function(a){return p.ready.promise().done(a),this},eq:function(a){return a=+a,a===-1?this.slice(a):this.slice(a,a+1)},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},slice:function(){return this.pushStack(k.apply(this,arguments),"slice",k.call(arguments).join(","))},map:function(a){return this.pushStack(p.map(this,function(b,c){return a.call(b,c,b)}))},end:function(){return this.prevObject||this.constructor(null)},push:j,sort:[].sort,splice:[].splice},p.fn.init.prototype=p.fn,p.extend=p.fn.extend=function(){var a,c,d,e,f,g,h=arguments[0]||{},i=1,j=arguments.length,k=!1;typeof h=="boolean"&&(k=h,h=arguments[1]||{},i=2),typeof h!="object"&&!p.isFunction(h)&&(h={}),j===i&&(h=this,--i);for(;i<j;i++)if((a=arguments[i])!=null)for(c in a){d=h[c],e=a[c];if(h===e)continue;k&&e&&(p.isPlainObject(e)||(f=p.isArray(e)))?(f?(f=!1,g=d&&p.isArray(d)?d:[]):g=d&&p.isPlainObject(d)?d:{},h[c]=p.extend(k,g,e)):e!==b&&(h[c]=e)}return h},p.extend({noConflict:function(b){return a.$===p&&(a.$=i),b&&a.jQuery===p&&(a.jQuery=h),p},isReady:!1,readyWait:1,holdReady:function(a){a?p.readyWait++:p.ready(!0)},ready:function(a){if(a===!0?--p.readyWait:p.isReady)return;if(!e.body)return setTimeout(p.ready,1);p.isReady=!0;if(a!==!0&&--p.readyWait>0)return;d.resolveWith(e,[p]),p.fn.trigger&&p(e).trigger("ready").off("ready")},isFunction:function(a){return p.type(a)==="function"},isArray:Array.isArray||function(a){return p.type(a)==="array"},isWindow:function(a){return a!=null&&a==a.window},isNumeric:function(a){return!isNaN(parseFloat(a))&&isFinite(a)},type:function(a){return a==null?String(a):E[m.call(a)]||"object"},isPlainObject:function(a){if(!a||p.type(a)!=="object"||a.nodeType||p.isWindow(a))return!1;try{if(a.constructor&&!n.call(a,"constructor")&&!n.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}var d;for(d in a);return d===b||n.call(a,d)},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},error:function(a){throw new Error(a)},parseHTML:function(a,b,c){var d;return!a||typeof a!="string"?null:(typeof b=="boolean"&&(c=b,b=0),b=b||e,(d=v.exec(a))?[b.createElement(d[1])]:(d=p.buildFragment([a],b,c?null:[]),p.merge([],(d.cacheable?p.clone(d.fragment):d.fragment).childNodes)))},parseJSON:function(b){if(!b||typeof b!="string")return null;b=p.trim(b);if(a.JSON&&a.JSON.parse)return a.JSON.parse(b);if(w.test(b.replace(y,"@").replace(z,"]").replace(x,"")))return(new Function("return "+b))();p.error("Invalid JSON: "+b)},parseXML:function(c){var d,e;if(!c||typeof c!="string")return null;try{a.DOMParser?(e=new DOMParser,d=e.parseFromString(c,"text/xml")):(d=new ActiveXObject("Microsoft.XMLDOM"),d.async="false",d.loadXML(c))}catch(f){d=b}return(!d||!d.documentElement||d.getElementsByTagName("parsererror").length)&&p.error("Invalid XML: "+c),d},noop:function(){},globalEval:function(b){b&&r.test(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(A,"ms-").replace(B,C)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toUpperCase()===b.toUpperCase()},each:function(a,c,d){var e,f=0,g=a.length,h=g===b||p.isFunction(a);if(d){if(h){for(e in a)if(c.apply(a[e],d)===!1)break}else for(;f<g;)if(c.apply(a[f++],d)===!1)break}else if(h){for(e in a)if(c.call(a[e],e,a[e])===!1)break}else for(;f<g;)if(c.call(a[f],f,a[f++])===!1)break;return a},trim:o?function(a){return a==null?"":o.call(a)}:function(a){return a==null?"":a.toString().replace(t,"")},makeArray:function(a,b){var c,d=b||[];return a!=null&&(c=p.type(a),a.length==null||c==="string"||c==="function"||c==="regexp"||p.isWindow(a)?j.call(d,a):p.merge(d,a)),d},inArray:function(a,b,c){var d;if(b){if(l)return l.call(b,a,c);d=b.length,c=c?c<0?Math.max(0,d+c):c:0;for(;c<d;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,c){var d=c.length,e=a.length,f=0;if(typeof d=="number")for(;f<d;f++)a[e++]=c[f];else while(c[f]!==b)a[e++]=c[f++];return a.length=e,a},grep:function(a,b,c){var d,e=[],f=0,g=a.length;c=!!c;for(;f<g;f++)d=!!b(a[f],f),c!==d&&e.push(a[f]);return e},map:function(a,c,d){var e,f,g=[],h=0,i=a.length,j=a instanceof p||i!==b&&typeof i=="number"&&(i>0&&a[0]&&a[i-1]||i===0||p.isArray(a));if(j)for(;h<i;h++)e=c(a[h],h,d),e!=null&&(g[g.length]=e);else for(f in a)e=c(a[f],f,d),e!=null&&(g[g.length]=e);return g.concat.apply([],g)},guid:1,proxy:function(a,c){var d,e,f;return typeof c=="string"&&(d=a[c],c=a,a=d),p.isFunction(a)?(e=k.call(arguments,2),f=function(){return a.apply(c,e.concat(k.call(arguments)))},f.guid=a.guid=a.guid||f.guid||p.guid++,f):b},access:function(a,c,d,e,f,g,h){var i,j=d==null,k=0,l=a.length;if(d&&typeof d=="object"){for(k in d)p.access(a,c,k,d[k],1,g,e);f=1}else if(e!==b){i=h===b&&p.isFunction(e),j&&(i?(i=c,c=function(a,b,c){return i.call(p(a),c)}):(c.call(a,e),c=null));if(c)for(;k<l;k++)c(a[k],d,i?e.call(a[k],k,c(a[k],d)):e,h);f=1}return f?a:j?c.call(a):l?c(a[0],d):g},now:function(){return(new Date).getTime()}}),p.ready.promise=function(b){if(!d){d=p.Deferred();if(e.readyState==="complete"||e.readyState!=="loading"&&e.addEventListener)setTimeout(p.ready,1);else if(e.addEventListener)e.addEventListener("DOMContentLoaded",D,!1),a.addEventListener("load",p.ready,!1);else{e.attachEvent("onreadystatechange",D),a.attachEvent("onload",p.ready);var c=!1;try{c=a.frameElement==null&&e.documentElement}catch(f){}c&&c.doScroll&&function g(){if(!p.isReady){try{c.doScroll("left")}catch(a){return setTimeout(g,50)}p.ready()}}()}}return d.promise(b)},p.each("Boolean Number String Function Array Date RegExp Object".split(" "),function(a,b){E["[object "+b+"]"]=b.toLowerCase()}),c=p(e);var F={};p.Callbacks=function(a){a=typeof a=="string"?F[a]||G(a):p.extend({},a);var c,d,e,f,g,h,i=[],j=!a.once&&[],k=function(b){c=a.memory&&b,d=!0,h=f||0,f=0,g=i.length,e=!0;for(;i&&h<g;h++)if(i[h].apply(b[0],b[1])===!1&&a.stopOnFalse){c=!1;break}e=!1,i&&(j?j.length&&k(j.shift()):c?i=[]:l.disable())},l={add:function(){if(i){var b=i.length;(function d(b){p.each(b,function(b,c){p.isFunction(c)&&(!a.unique||!l.has(c))?i.push(c):c&&c.length&&d(c)})})(arguments),e?g=i.length:c&&(f=b,k(c))}return this},remove:function(){return i&&p.each(arguments,function(a,b){var c;while((c=p.inArray(b,i,c))>-1)i.splice(c,1),e&&(c<=g&&g--,c<=h&&h--)}),this},has:function(a){return p.inArray(a,i)>-1},empty:function(){return i=[],this},disable:function(){return i=j=c=b,this},disabled:function(){return!i},lock:function(){return j=b,c||l.disable(),this},locked:function(){return!j},fireWith:function(a,b){return b=b||[],b=[a,b.slice?b.slice():b],i&&(!d||j)&&(e?j.push(b):k(b)),this},fire:function(){return l.fireWith(this,arguments),this},fired:function(){return!!d}};return l},p.extend({Deferred:function(a){var b=[["resolve","done",p.Callbacks("once memory"),"resolved"],["reject","fail",p.Callbacks("once memory"),"rejected"],["notify","progress",p.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return p.Deferred(function(c){p.each(b,function(b,d){var f=d[0],g=a[b];e[d[1]](p.isFunction(g)?function(){var a=g.apply(this,arguments);a&&p.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f+"With"](this===e?c:this,[a])}:c[f])}),a=null}).promise()},promise:function(a){return typeof a=="object"?p.extend(a,d):d}},e={};return d.pipe=d.then,p.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[a^1][2].disable,b[2][2].lock),e[f[0]]=g.fire,e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=k.call(arguments),d=c.length,e=d!==1||a&&p.isFunction(a.promise)?d:0,f=e===1?a:p.Deferred(),g=function(a,b,c){return function(d){b[a]=this,c[a]=arguments.length>1?k.call(arguments):d,c===h?f.notifyWith(b,c):--e||f.resolveWith(b,c)}},h,i,j;if(d>1){h=new Array(d),i=new Array(d),j=new Array(d);for(;b<d;b++)c[b]&&p.isFunction(c[b].promise)?c[b].promise().done(g(b,j,c)).fail(f.reject).progress(g(b,i,h)):--e}return e||f.resolveWith(j,c),f.promise()}}),p.support=function(){var b,c,d,f,g,h,i,j,k,l,m,n=e.createElement("div");n.setAttribute("className","t"),n.innerHTML=" <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",c=n.getElementsByTagName("*"),d=n.getElementsByTagName("a")[0],d.style.cssText="top:1px;float:left;opacity:.5";if(!c||!c.length||!d)return{};f=e.createElement("select"),g=f.appendChild(e.createElement("option")),h=n.getElementsByTagName("input")[0],b={leadingWhitespace:n.firstChild.nodeType===3,tbody:!n.getElementsByTagName("tbody").length,htmlSerialize:!!n.getElementsByTagName("link").length,style:/top/.test(d.getAttribute("style")),hrefNormalized:d.getAttribute("href")==="/a",opacity:/^0.5/.test(d.style.opacity),cssFloat:!!d.style.cssFloat,checkOn:h.value==="on",optSelected:g.selected,getSetAttribute:n.className!=="t",enctype:!!e.createElement("form").enctype,html5Clone:e.createElement("nav").cloneNode(!0).outerHTML!=="<:nav></:nav>",boxModel:e.compatMode==="CSS1Compat",submitBubbles:!0,changeBubbles:!0,focusinBubbles:!1,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},h.checked=!0,b.noCloneChecked=h.cloneNode(!0).checked,f.disabled=!0,b.optDisabled=!g.disabled;try{delete n.test}catch(o){b.deleteExpando=!1}!n.addEventListener&&n.attachEvent&&n.fireEvent&&(n.attachEvent("onclick",m=function(){b.noCloneEvent=!1}),n.cloneNode(!0).fireEvent("onclick"),n.detachEvent("onclick",m)),h=e.createElement("input"),h.value="t",h.setAttribute("type","radio"),b.radioValue=h.value==="t",h.setAttribute("checked","checked"),h.setAttribute("name","t"),n.appendChild(h),i=e.createDocumentFragment(),i.appendChild(n.lastChild),b.checkClone=i.cloneNode(!0).cloneNode(!0).lastChild.checked,b.appendChecked=h.checked,i.removeChild(h),i.appendChild(n);if(n.attachEvent)for(k in{submit:!0,change:!0,focusin:!0})j="on"+k,l=j in n,l||(n.setAttribute(j,"return;"),l=typeof n[j]=="function"),b[k+"Bubbles"]=l;return p(function(){var c,d,f,g,h="padding:0;margin:0;border:0;display:block;overflow:hidden;",i=e.getElementsByTagName("body")[0];if(!i)return;c=e.createElement("div"),c.style.cssText="visibility:hidden;border:0;width:0;height:0;position:static;top:0;margin-top:1px",i.insertBefore(c,i.firstChild),d=e.createElement("div"),c.appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",f=d.getElementsByTagName("td"),f[0].style.cssText="padding:0;margin:0;border:0;display:none",l=f[0].offsetHeight===0,f[0].style.display="",f[1].style.display="none",b.reliableHiddenOffsets=l&&f[0].offsetHeight===0,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1%;",b.boxSizing=d.offsetWidth===4,b.doesNotIncludeMarginInBodyOffset=i.offsetTop!==1,a.getComputedStyle&&(b.pixelPosition=(a.getComputedStyle(d,null)||{}).top!=="1%",b.boxSizingReliable=(a.getComputedStyle(d,null)||{width:"4px"}).width==="4px",g=e.createElement("div"),g.style.cssText=d.style.cssText=h,g.style.marginRight=g.style.width="0",d.style.width="1px",d.appendChild(g),b.reliableMarginRight=!parseFloat((a.getComputedStyle(g,null)||{}).marginRight)),typeof d.style.zoom!="undefined"&&(d.innerHTML="",d.style.cssText=h+"width:1px;padding:1px;display:inline;zoom:1",b.inlineBlockNeedsLayout=d.offsetWidth===3,d.style.display="block",d.style.overflow="visible",d.innerHTML="<div></div>",d.firstChild.style.width="5px",b.shrinkWrapBlocks=d.offsetWidth!==3,c.style.zoom=1),i.removeChild(c),c=d=f=g=null}),i.removeChild(n),c=d=f=g=h=i=n=null,b}();var H=/^(?:\{.*\}|\[.*\])$/,I=/([A-Z])/g;p.extend({cache:{},deletedIds:[],uuid:0,expando:"jQuery"+(p.fn.jquery+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(a){return a=a.nodeType?p.cache[a[p.expando]]:a[p.expando],!!a&&!K(a)},data:function(a,c,d,e){if(!p.acceptData(a))return;var f,g,h=p.expando,i=typeof c=="string",j=a.nodeType,k=j?p.cache:a,l=j?a[h]:a[h]&&h;if((!l||!k[l]||!e&&!k[l].data)&&i&&d===b)return;l||(j?a[h]=l=p.deletedIds.pop()||++p.uuid:l=h),k[l]||(k[l]={},j||(k[l].toJSON=p.noop));if(typeof c=="object"||typeof c=="function")e?k[l]=p.extend(k[l],c):k[l].data=p.extend(k[l].data,c);return f=k[l],e||(f.data||(f.data={}),f=f.data),d!==b&&(f[p.camelCase(c)]=d),i?(g=f[c],g==null&&(g=f[p.camelCase(c)])):g=f,g},removeData:function(a,b,c){if(!p.acceptData(a))return;var d,e,f,g=a.nodeType,h=g?p.cache:a,i=g?a[p.expando]:p.expando;if(!h[i])return;if(b){d=c?h[i]:h[i].data;if(d){p.isArray(b)||(b in d?b=[b]:(b=p.camelCase(b),b in d?b=[b]:b=b.split(" ")));for(e=0,f=b.length;e<f;e++)delete d[b[e]];if(!(c?K:p.isEmptyObject)(d))return}}if(!c){delete h[i].data;if(!K(h[i]))return}g?p.cleanData([a],!0):p.support.deleteExpando||h!=h.window?delete h[i]:h[i]=null},_data:function(a,b,c){return p.data(a,b,c,!0)},acceptData:function(a){var b=a.nodeName&&p.noData[a.nodeName.toLowerCase()];return!b||b!==!0&&a.getAttribute("classid")===b}}),p.fn.extend({data:function(a,c){var d,e,f,g,h,i=this[0],j=0,k=null;if(a===b){if(this.length){k=p.data(i);if(i.nodeType===1&&!p._data(i,"parsedAttrs")){f=i.attributes;for(h=f.length;j<h;j++)g=f[j].name,g.indexOf("data-")===0&&(g=p.camelCase(g.substring(5)),J(i,g,k[g]));p._data(i,"parsedAttrs",!0)}}return k}return typeof a=="object"?this.each(function(){p.data(this,a)}):(d=a.split(".",2),d[1]=d[1]?"."+d[1]:"",e=d[1]+"!",p.access(this,function(c){if(c===b)return k=this.triggerHandler("getData"+e,[d[0]]),k===b&&i&&(k=p.data(i,a),k=J(i,a,k)),k===b&&d[1]?this.data(d[0]):k;d[1]=c,this.each(function(){var b=p(this);b.triggerHandler("setData"+e,d),p.data(this,a,c),b.triggerHandler("changeData"+e,d)})},null,c,arguments.length>1,null,!1))},removeData:function(a){return this.each(function(){p.removeData(this,a)})}}),p.extend({queue:function(a,b,c){var d;if(a)return b=(b||"fx")+"queue",d=p._data(a,b),c&&(!d||p.isArray(c)?d=p._data(a,b,p.makeArray(c)):d.push(c)),d||[]},dequeue:function(a,b){b=b||"fx";var c=p.queue(a,b),d=c.shift(),e=p._queueHooks(a,b),f=function(){p.dequeue(a,b)};d==="inprogress"&&(d=c.shift()),d&&(b==="fx"&&c.unshift("inprogress"),delete e.stop,d.call(a,f,e)),!c.length&&e&&e.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return p._data(a,c)||p._data(a,c,{empty:p.Callbacks("once memory").add(function(){p.removeData(a,b+"queue",!0),p.removeData(a,c,!0)})})}}),p.fn.extend({queue:function(a,c){var d=2;return typeof a!="string"&&(c=a,a="fx",d--),arguments.length<d?p.queue(this[0],a):c===b?this:this.each(function(){var b=p.queue(this,a,c);p._queueHooks(this,a),a==="fx"&&b[0]!=="inprogress"&&p.dequeue(this,a)})},dequeue:function(a){return this.each(function(){p.dequeue(this,a)})},delay:function(a,b){return a=p.fx?p.fx.speeds[a]||a:a,b=b||"fx",this.queue(b,function(b,c){var d=setTimeout(b,a);c.stop=function(){clearTimeout(d)}})},clearQueue:function(a){return this.queue(a||"fx",[])},promise:function(a,c){var d,e=1,f=p.Deferred(),g=this,h=this.length,i=function(){--e||f.resolveWith(g,[g])};typeof a!="string"&&(c=a,a=b),a=a||"fx";while(h--)(d=p._data(g[h],a+"queueHooks"))&&d.empty&&(e++,d.empty.add(i));return i(),f.promise(c)}});var L,M,N,O=/[\t\r\n]/g,P=/\r/g,Q=/^(?:button|input)$/i,R=/^(?:button|input|object|select|textarea)$/i,S=/^a(?:rea|)$/i,T=/^(?:autofocus|autoplay|async|checked|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped|selected)$/i,U=p.support.getSetAttribute;p.fn.extend({attr:function(a,b){return p.access(this,p.attr,a,b,arguments.length>1)},removeAttr:function(a){return this.each(function(){p.removeAttr(this,a)})},prop:function(a,b){return p.access(this,p.prop,a,b,arguments.length>1)},removeProp:function(a){return a=p.propFix[a]||a,this.each(function(){try{this[a]=b,delete this[a]}catch(c){}})},addClass:function(a){var b,c,d,e,f,g,h;if(p.isFunction(a))return this.each(function(b){p(this).addClass(a.call(this,b,this.className))});if(a&&typeof a=="string"){b=a.split(s);for(c=0,d=this.length;c<d;c++){e=this[c];if(e.nodeType===1)if(!e.className&&b.length===1)e.className=a;else{f=" "+e.className+" ";for(g=0,h=b.length;g<h;g++)~f.indexOf(" "+b[g]+" ")||(f+=b[g]+" ");e.className=p.trim(f)}}}return this},removeClass:function(a){var c,d,e,f,g,h,i;if(p.isFunction(a))return this.each(function(b){p(this).removeClass(a.call(this,b,this.className))});if(a&&typeof a=="string"||a===b){c=(a||"").split(s);for(h=0,i=this.length;h<i;h++){e=this[h];if(e.nodeType===1&&e.className){d=(" "+e.className+" ").replace(O," ");for(f=0,g=c.length;f<g;f++)while(d.indexOf(" "+c[f]+" ")>-1)d=d.replace(" "+c[f]+" "," ");e.className=a?p.trim(d):""}}}return this},toggleClass:function(a,b){var c=typeof a,d=typeof b=="boolean";return p.isFunction(a)?this.each(function(c){p(this).toggleClass(a.call(this,c,this.className,b),b)}):this.each(function(){if(c==="string"){var e,f=0,g=p(this),h=b,i=a.split(s);while(e=i[f++])h=d?h:!g.hasClass(e),g[h?"addClass":"removeClass"](e)}else if(c==="undefined"||c==="boolean")this.className&&p._data(this,"__className__",this.className),this.className=this.className||a===!1?"":p._data(this,"__className__")||""})},hasClass:function(a){var b=" "+a+" ",c=0,d=this.length;for(;c<d;c++)if(this[c].nodeType===1&&(" "+this[c].className+" ").replace(O," ").indexOf(b)>-1)return!0;return!1},val:function(a){var c,d,e,f=this[0];if(!arguments.length){if(f)return c=p.valHooks[f.type]||p.valHooks[f.nodeName.toLowerCase()],c&&"get"in c&&(d=c.get(f,"value"))!==b?d:(d=f.value,typeof d=="string"?d.replace(P,""):d==null?"":d);return}return e=p.isFunction(a),this.each(function(d){var f,g=p(this);if(this.nodeType!==1)return;e?f=a.call(this,d,g.val()):f=a,f==null?f="":typeof f=="number"?f+="":p.isArray(f)&&(f=p.map(f,function(a){return a==null?"":a+""})),c=p.valHooks[this.type]||p.valHooks[this.nodeName.toLowerCase()];if(!c||!("set"in c)||c.set(this,f,"value")===b)this.value=f})}}),p.extend({valHooks:{option:{get:function(a){var b=a.attributes.value;return!b||b.specified?a.value:a.text}},select:{get:function(a){var b,c,d,e,f=a.selectedIndex,g=[],h=a.options,i=a.type==="select-one";if(f<0)return null;c=i?f:0,d=i?f+1:h.length;for(;c<d;c++){e=h[c];if(e.selected&&(p.support.optDisabled?!e.disabled:e.getAttribute("disabled")===null)&&(!e.parentNode.disabled||!p.nodeName(e.parentNode,"optgroup"))){b=p(e).val();if(i)return b;g.push(b)}}return i&&!g.length&&h.length?p(h[f]).val():g},set:function(a,b){var c=p.makeArray(b);return p(a).find("option").each(function(){this.selected=p.inArray(p(this).val(),c)>=0}),c.length||(a.selectedIndex=-1),c}}},attrFn:{},attr:function(a,c,d,e){var f,g,h,i=a.nodeType;if(!a||i===3||i===8||i===2)return;if(e&&p.isFunction(p.fn[c]))return p(a)[c](d);if(typeof a.getAttribute=="undefined")return p.prop(a,c,d);h=i!==1||!p.isXMLDoc(a),h&&(c=c.toLowerCase(),g=p.attrHooks[c]||(T.test(c)?M:L));if(d!==b){if(d===null){p.removeAttr(a,c);return}return g&&"set"in g&&h&&(f=g.set(a,d,c))!==b?f:(a.setAttribute(c,""+d),d)}return g&&"get"in g&&h&&(f=g.get(a,c))!==null?f:(f=a.getAttribute(c),f===null?b:f)},removeAttr:function(a,b){var c,d,e,f,g=0;if(b&&a.nodeType===1){d=b.split(s);for(;g<d.length;g++)e=d[g],e&&(c=p.propFix[e]||e,f=T.test(e),f||p.attr(a,e,""),a.removeAttribute(U?e:c),f&&c in a&&(a[c]=!1))}},attrHooks:{type:{set:function(a,b){if(Q.test(a.nodeName)&&a.parentNode)p.error("type property can't be changed");else if(!p.support.radioValue&&b==="radio"&&p.nodeName(a,"input")){var c=a.value;return a.setAttribute("type",b),c&&(a.value=c),b}}},value:{get:function(a,b){return L&&p.nodeName(a,"button")?L.get(a,b):b in a?a.value:null},set:function(a,b,c){if(L&&p.nodeName(a,"button"))return L.set(a,b,c);a.value=b}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",colspan:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(a,c,d){var e,f,g,h=a.nodeType;if(!a||h===3||h===8||h===2)return;return g=h!==1||!p.isXMLDoc(a),g&&(c=p.propFix[c]||c,f=p.propHooks[c]),d!==b?f&&"set"in f&&(e=f.set(a,d,c))!==b?e:a[c]=d:f&&"get"in f&&(e=f.get(a,c))!==null?e:a[c]},propHooks:{tabIndex:{get:function(a){var c=a.getAttributeNode("tabindex");return c&&c.specified?parseInt(c.value,10):R.test(a.nodeName)||S.test(a.nodeName)&&a.href?0:b}}}}),M={get:function(a,c){var d,e=p.prop(a,c);return e===!0||typeof e!="boolean"&&(d=a.getAttributeNode(c))&&d.nodeValue!==!1?c.toLowerCase():b},set:function(a,b,c){var d;return b===!1?p.removeAttr(a,c):(d=p.propFix[c]||c,d in a&&(a[d]=!0),a.setAttribute(c,c.toLowerCase())),c}},U||(N={name:!0,id:!0,coords:!0},L=p.valHooks.button={get:function(a,c){var d;return d=a.getAttributeNode(c),d&&(N[c]?d.value!=="":d.specified)?d.value:b},set:function(a,b,c){var d=a.getAttributeNode(c);return d||(d=e.createAttribute(c),a.setAttributeNode(d)),d.value=b+""}},p.each(["width","height"],function(a,b){p.attrHooks[b]=p.extend(p.attrHooks[b],{set:function(a,c){if(c==="")return a.setAttribute(b,"auto"),c}})}),p.attrHooks.contenteditable={get:L.get,set:function(a,b,c){b===""&&(b="false"),L.set(a,b,c)}}),p.support.hrefNormalized||p.each(["href","src","width","height"],function(a,c){p.attrHooks[c]=p.extend(p.attrHooks[c],{get:function(a){var d=a.getAttribute(c,2);return d===null?b:d}})}),p.support.style||(p.attrHooks.style={get:function(a){return a.style.cssText.toLowerCase()||b},set:function(a,b){return a.style.cssText=""+b}}),p.support.optSelected||(p.propHooks.selected=p.extend(p.propHooks.selected,{get:function(a){var b=a.parentNode;return b&&(b.selectedIndex,b.parentNode&&b.parentNode.selectedIndex),null}})),p.support.enctype||(p.propFix.enctype="encoding"),p.support.checkOn||p.each(["radio","checkbox"],function(){p.valHooks[this]={get:function(a){return a.getAttribute("value")===null?"on":a.value}}}),p.each(["radio","checkbox"],function(){p.valHooks[this]=p.extend(p.valHooks[this],{set:function(a,b){if(p.isArray(b))return a.checked=p.inArray(p(a).val(),b)>=0}})});var V=/^(?:textarea|input|select)$/i,W=/^([^\.]*|)(?:\.(.+)|)$/,X=/(?:^|\s)hover(\.\S+|)\b/,Y=/^key/,Z=/^(?:mouse|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=function(a){return p.event.special.hover?a:a.replace(X,"mouseenter$1 mouseleave$1")};p.event={add:function(a,c,d,e,f){var g,h,i,j,k,l,m,n,o,q,r;if(a.nodeType===3||a.nodeType===8||!c||!d||!(g=p._data(a)))return;d.handler&&(o=d,d=o.handler,f=o.selector),d.guid||(d.guid=p.guid++),i=g.events,i||(g.events=i={}),h=g.handle,h||(g.handle=h=function(a){return typeof p!="undefined"&&(!a||p.event.triggered!==a.type)?p.event.dispatch.apply(h.elem,arguments):b},h.elem=a),c=p.trim(_(c)).split(" ");for(j=0;j<c.length;j++){k=W.exec(c[j])||[],l=k[1],m=(k[2]||"").split(".").sort(),r=p.event.special[l]||{},l=(f?r.delegateType:r.bindType)||l,r=p.event.special[l]||{},n=p.extend({type:l,origType:k[1],data:e,handler:d,guid:d.guid,selector:f,namespace:m.join(".")},o),q=i[l];if(!q){q=i[l]=[],q.delegateCount=0;if(!r.setup||r.setup.call(a,e,m,h)===!1)a.addEventListener?a.addEventListener(l,h,!1):a.attachEvent&&a.attachEvent("on"+l,h)}r.add&&(r.add.call(a,n),n.handler.guid||(n.handler.guid=d.guid)),f?q.splice(q.delegateCount++,0,n):q.push(n),p.event.global[l]=!0}a=null},global:{},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,n,o,q,r=p.hasData(a)&&p._data(a);if(!r||!(m=r.events))return;b=p.trim(_(b||"")).split(" ");for(f=0;f<b.length;f++){g=W.exec(b[f])||[],h=i=g[1],j=g[2];if(!h){for(h in m)p.event.remove(a,h+b[f],c,d,!0);continue}n=p.event.special[h]||{},h=(d?n.delegateType:n.bindType)||h,o=m[h]||[],k=o.length,j=j?new RegExp("(^|\\.)"+j.split(".").sort().join("\\.(?:.*\\.|)")+"(\\.|$)"):null;for(l=0;l<o.length;l++)q=o[l],(e||i===q.origType)&&(!c||c.guid===q.guid)&&(!j||j.test(q.namespace))&&(!d||d===q.selector||d==="**"&&q.selector)&&(o.splice(l--,1),q.selector&&o.delegateCount--,n.remove&&n.remove.call(a,q));o.length===0&&k!==o.length&&((!n.teardown||n.teardown.call(a,j,r.handle)===!1)&&p.removeEvent(a,h,r.handle),delete m[h])}p.isEmptyObject(m)&&(delete r.handle,p.removeData(a,"events",!0))},customEvent:{getData:!0,setData:!0,changeData:!0},trigger:function(c,d,f,g){if(!f||f.nodeType!==3&&f.nodeType!==8){var h,i,j,k,l,m,n,o,q,r,s=c.type||c,t=[];if($.test(s+p.event.triggered))return;s.indexOf("!")>=0&&(s=s.slice(0,-1),i=!0),s.indexOf(".")>=0&&(t=s.split("."),s=t.shift(),t.sort());if((!f||p.event.customEvent[s])&&!p.event.global[s])return;c=typeof c=="object"?c[p.expando]?c:new p.Event(s,c):new p.Event(s),c.type=s,c.isTrigger=!0,c.exclusive=i,c.namespace=t.join("."),c.namespace_re=c.namespace?new RegExp("(^|\\.)"+t.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,m=s.indexOf(":")<0?"on"+s:"";if(!f){h=p.cache;for(j in h)h[j].events&&h[j].events[s]&&p.event.trigger(c,d,h[j].handle.elem,!0);return}c.result=b,c.target||(c.target=f),d=d!=null?p.makeArray(d):[],d.unshift(c),n=p.event.special[s]||{};if(n.trigger&&n.trigger.apply(f,d)===!1)return;q=[[f,n.bindType||s]];if(!g&&!n.noBubble&&!p.isWindow(f)){r=n.delegateType||s,k=$.test(r+s)?f:f.parentNode;for(l=f;k;k=k.parentNode)q.push([k,r]),l=k;l===(f.ownerDocument||e)&&q.push([l.defaultView||l.parentWindow||a,r])}for(j=0;j<q.length&&!c.isPropagationStopped();j++)k=q[j][0],c.type=q[j][1],o=(p._data(k,"events")||{})[c.type]&&p._data(k,"handle"),o&&o.apply(k,d),o=m&&k[m],o&&p.acceptData(k)&&o.apply(k,d)===!1&&c.preventDefault();return c.type=s,!g&&!c.isDefaultPrevented()&&(!n._default||n._default.apply(f.ownerDocument,d)===!1)&&(s!=="click"||!p.nodeName(f,"a"))&&p.acceptData(f)&&m&&f[s]&&(s!=="focus"&&s!=="blur"||c.target.offsetWidth!==0)&&!p.isWindow(f)&&(l=f[m],l&&(f[m]=null),p.event.triggered=s,f[s](),p.event.triggered=b,l&&(f[m]=l)),c.result}return},dispatch:function(c){c=p.event.fix(c||a.event);var d,e,f,g,h,i,j,k,l,m,n,o=(p._data(this,"events")||{})[c.type]||[],q=o.delegateCount,r=[].slice.call(arguments),s=!c.exclusive&&!c.namespace,t=p.event.special[c.type]||{},u=[];r[0]=c,c.delegateTarget=this;if(t.preDispatch&&t.preDispatch.call(this,c)===!1)return;if(q&&(!c.button||c.type!=="click")){g=p(this),g.context=this;for(f=c.target;f!=this;f=f.parentNode||this)if(f.disabled!==!0||c.type!=="click"){i={},k=[],g[0]=f;for(d=0;d<q;d++)l=o[d],m=l.selector,i[m]===b&&(i[m]=g.is(m)),i[m]&&k.push(l);k.length&&u.push({elem:f,matches:k})}}o.length>q&&u.push({elem:this,matches:o.slice(q)});for(d=0;d<u.length&&!c.isPropagationStopped();d++){j=u[d],c.currentTarget=j.elem;for(e=0;e<j.matches.length&&!c.isImmediatePropagationStopped();e++){l=j.matches[e];if(s||!c.namespace&&!l.namespace||c.namespace_re&&c.namespace_re.test(l.namespace))c.data=l.data,c.handleObj=l,h=((p.event.special[l.origType]||{}).handle||l.handler).apply(j.elem,r),h!==b&&(c.result=h,h===!1&&(c.preventDefault(),c.stopPropagation()))}}return t.postDispatch&&t.postDispatch.call(this,c),c.result},props:"attrChange attrName relatedNode srcElement altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(a,b){return a.which==null&&(a.which=b.charCode!=null?b.charCode:b.keyCode),a}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(a,c){var d,f,g,h=c.button,i=c.fromElement;return a.pageX==null&&c.clientX!=null&&(d=a.target.ownerDocument||e,f=d.documentElement,g=d.body,a.pageX=c.clientX+(f&&f.scrollLeft||g&&g.scrollLeft||0)-(f&&f.clientLeft||g&&g.clientLeft||0),a.pageY=c.clientY+(f&&f.scrollTop||g&&g.scrollTop||0)-(f&&f.clientTop||g&&g.clientTop||0)),!a.relatedTarget&&i&&(a.relatedTarget=i===a.target?c.toElement:i),!a.which&&h!==b&&(a.which=h&1?1:h&2?3:h&4?2:0),a}},fix:function(a){if(a[p.expando])return a;var b,c,d=a,f=p.event.fixHooks[a.type]||{},g=f.props?this.props.concat(f.props):this.props;a=p.Event(d);for(b=g.length;b;)c=g[--b],a[c]=d[c];return a.target||(a.target=d.srcElement||e),a.target.nodeType===3&&(a.target=a.target.parentNode),a.metaKey=!!a.metaKey,f.filter?f.filter(a,d):a},special:{ready:{setup:p.bindReady},load:{noBubble:!0},focus:{delegateType:"focusin"},blur:{delegateType:"focusout"},beforeunload:{setup:function(a,b,c){p.isWindow(this)&&(this.onbeforeunload=c)},teardown:function(a,b){this.onbeforeunload===b&&(this.onbeforeunload=null)}}},simulate:function(a,b,c,d){var e=p.extend(new p.Event,c,{type:a,isSimulated:!0,originalEvent:{}});d?p.event.trigger(e,null,b):p.event.dispatch.call(b,e),e.isDefaultPrevented()&&c.preventDefault()}},p.event.handle=p.event.dispatch,p.removeEvent=e.removeEventListener?function(a,b,c){a.removeEventListener&&a.removeEventListener(b,c,!1)}:function(a,b,c){var d="on"+b;a.detachEvent&&(typeof a[d]=="undefined"&&(a[d]=null),a.detachEvent(d,c))},p.Event=function(a,b){if(this instanceof p.Event)a&&a.type?(this.originalEvent=a,this.type=a.type,this.isDefaultPrevented=a.defaultPrevented||a.returnValue===!1||a.getPreventDefault&&a.getPreventDefault()?bb:ba):this.type=a,b&&p.extend(this,b),this.timeStamp=a&&a.timeStamp||p.now(),this[p.expando]=!0;else return new p.Event(a,b)},p.Event.prototype={preventDefault:function(){this.isDefaultPrevented=bb;var a=this.originalEvent;if(!a)return;a.preventDefault?a.preventDefault():a.returnValue=!1},stopPropagation:function(){this.isPropagationStopped=bb;var a=this.originalEvent;if(!a)return;a.stopPropagation&&a.stopPropagation(),a.cancelBubble=!0},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=bb,this.stopPropagation()},isDefaultPrevented:ba,isPropagationStopped:ba,isImmediatePropagationStopped:ba},p.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(a,b){p.event.special[a]={delegateType:b,bindType:b,handle:function(a){var c,d=this,e=a.relatedTarget,f=a.handleObj,g=f.selector;if(!e||e!==d&&!p.contains(d,e))a.type=f.origType,c=f.handler.apply(this,arguments),a.type=b;return c}}}),p.support.submitBubbles||(p.event.special.submit={setup:function(){if(p.nodeName(this,"form"))return!1;p.event.add(this,"click._submit keypress._submit",function(a){var c=a.target,d=p.nodeName(c,"input")||p.nodeName(c,"button")?c.form:b;d&&!p._data(d,"_submit_attached")&&(p.event.add(d,"submit._submit",function(a){a._submit_bubble=!0}),p._data(d,"_submit_attached",!0))})},postDispatch:function(a){a._submit_bubble&&(delete a._submit_bubble,this.parentNode&&!a.isTrigger&&p.event.simulate("submit",this.parentNode,a,!0))},teardown:function(){if(p.nodeName(this,"form"))return!1;p.event.remove(this,"._submit")}}),p.support.changeBubbles||(p.event.special.change={setup:function(){if(V.test(this.nodeName)){if(this.type==="checkbox"||this.type==="radio")p.event.add(this,"propertychange._change",function(a){a.originalEvent.propertyName==="checked"&&(this._just_changed=!0)}),p.event.add(this,"click._change",function(a){this._just_changed&&!a.isTrigger&&(this._just_changed=!1),p.event.simulate("change",this,a,!0)});return!1}p.event.add(this,"beforeactivate._change",function(a){var b=a.target;V.test(b.nodeName)&&!p._data(b,"_change_attached")&&(p.event.add(b,"change._change",function(a){this.parentNode&&!a.isSimulated&&!a.isTrigger&&p.event.simulate("change",this.parentNode,a,!0)}),p._data(b,"_change_attached",!0))})},handle:function(a){var b=a.target;if(this!==b||a.isSimulated||a.isTrigger||b.type!=="radio"&&b.type!=="checkbox")return a.handleObj.handler.apply(this,arguments)},teardown:function(){return p.event.remove(this,"._change"),V.test(this.nodeName)}}),p.support.focusinBubbles||p.each({focus:"focusin",blur:"focusout"},function(a,b){var c=0,d=function(a){p.event.simulate(b,a.target,p.event.fix(a),!0)};p.event.special[b]={setup:function(){c++===0&&e.addEventListener(a,d,!0)},teardown:function(){--c===0&&e.removeEventListener(a,d,!0)}}}),p.fn.extend({on:function(a,c,d,e,f){var g,h;if(typeof a=="object"){typeof c!="string"&&(d=d||c,c=b);for(h in a)this.on(h,c,d,a[h],f);return this}d==null&&e==null?(e=c,d=c=b):e==null&&(typeof c=="string"?(e=d,d=b):(e=d,d=c,c=b));if(e===!1)e=ba;else if(!e)return this;return f===1&&(g=e,e=function(a){return p().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=p.guid++)),this.each(function(){p.event.add(this,a,e,d,c)})},one:function(a,b,c,d){return this.on(a,b,c,d,1)},off:function(a,c,d){var e,f;if(a&&a.preventDefault&&a.handleObj)return e=a.handleObj,p(a.delegateTarget).off(e.namespace?e.origType+"."+e.namespace:e.origType,e.selector,e.handler),this;if(typeof a=="object"){for(f in a)this.off(f,c,a[f]);return this}if(c===!1||typeof c=="function")d=c,c=b;return d===!1&&(d=ba),this.each(function(){p.event.remove(this,a,d,c)})},bind:function(a,b,c){return this.on(a,null,b,c)},unbind:function(a,b){return this.off(a,null,b)},live:function(a,b,c){return p(this.context).on(a,this.selector,b,c),this},die:function(a,b){return p(this.context).off(a,this.selector||"**",b),this},delegate:function(a,b,c,d){return this.on(b,a,c,d)},undelegate:function(a,b,c){return arguments.length==1?this.off(a,"**"):this.off(b,a||"**",c)},trigger:function(a,b){return this.each(function(){p.event.trigger(a,b,this)})},triggerHandler:function(a,b){if(this[0])return p.event.trigger(a,b,this[0],!0)},toggle:function(a){var b=arguments,c=a.guid||p.guid++,d=0,e=function(c){var e=(p._data(this,"lastToggle"+a.guid)||0)%d;return p._data(this,"lastToggle"+a.guid,e+1),c.preventDefault(),b[e].apply(this,arguments)||!1};e.guid=c;while(d<b.length)b[d++].guid=c;return this.click(e)},hover:function(a,b){return this.mouseenter(a).mouseleave(b||a)}}),p.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(a,b){p.fn[b]=function(a,c){return c==null&&(c=a,a=null),arguments.length>0?this.on(b,null,a,c):this.trigger(b)},Y.test(b)&&(p.event.fixHooks[b]=p.event.keyHooks),Z.test(b)&&(p.event.fixHooks[b]=p.event.mouseHooks)}),function(a,b){function bd(a,b,c,d){var e=0,f=b.length;for(;e<f;e++)Z(a,b[e],c,d)}function be(a,b,c,d,e,f){var g,h=$.setFilters[b.toLowerCase()];return h||Z.error(b),(a||!(g=e))&&bd(a||"*",d,g=[],e),g.length>0?h(g,c,f):[]}function bf(a,c,d,e,f){var g,h,i,j,k,l,m,n,p=0,q=f.length,s=L.POS,t=new RegExp("^"+s.source+"(?!"+r+")","i"),u=function(){var a=1,c=arguments.length-2;for(;a<c;a++)arguments[a]===b&&(g[a]=b)};for(;p<q;p++){s.exec(""),a=f[p],j=[],i=0,k=e;while(g=s.exec(a)){n=s.lastIndex=g.index+g[0].length;if(n>i){m=a.slice(i,g.index),i=n,l=[c],B.test(m)&&(k&&(l=k),k=e);if(h=H.test(m))m=m.slice(0,-5).replace(B,"$&*");g.length>1&&g[0].replace(t,u),k=be(m,g[1],g[2],l,k,h)}}k?(j=j.concat(k),(m=a.slice(i))&&m!==")"?B.test(m)?bd(m,j,d,e):Z(m,c,d,e?e.concat(k):k):o.apply(d,j)):Z(a,c,d,e)}return q===1?d:Z.uniqueSort(d)}function bg(a,b,c){var d,e,f,g=[],i=0,j=D.exec(a),k=!j.pop()&&!j.pop(),l=k&&a.match(C)||[""],m=$.preFilter,n=$.filter,o=!c&&b!==h;for(;(e=l[i])!=null&&k;i++){g.push(d=[]),o&&(e=" "+e);while(e){k=!1;if(j=B.exec(e))e=e.slice(j[0].length),k=d.push({part:j.pop().replace(A," "),captures:j});for(f in n)(j=L[f].exec(e))&&(!m[f]||(j=m[f](j,b,c)))&&(e=e.slice(j.shift().length),k=d.push({part:f,captures:j}));if(!k)break}}return k||Z.error(a),g}function bh(a,b,e){var f=b.dir,g=m++;return a||(a=function(a){return a===e}),b.first?function(b,c){while(b=b[f])if(b.nodeType===1)return a(b,c)&&b}:function(b,e){var h,i=g+"."+d,j=i+"."+c;while(b=b[f])if(b.nodeType===1){if((h=b[q])===j)return b.sizset;if(typeof h=="string"&&h.indexOf(i)===0){if(b.sizset)return b}else{b[q]=j;if(a(b,e))return b.sizset=!0,b;b.sizset=!1}}}}function bi(a,b){return a?function(c,d){var e=b(c,d);return e&&a(e===!0?c:e,d)}:b}function bj(a,b,c){var d,e,f=0;for(;d=a[f];f++)$.relative[d.part]?e=bh(e,$.relative[d.part],b):(d.captures.push(b,c),e=bi(e,$.filter[d.part].apply(null,d.captures)));return e}function bk(a){return function(b,c){var d,e=0;for(;d=a[e];e++)if(d(b,c))return!0;return!1}}var c,d,e,f,g,h=a.document,i=h.documentElement,j="undefined",k=!1,l=!0,m=0,n=[].slice,o=[].push,q=("sizcache"+Math.random()).replace(".",""),r="[\\x20\\t\\r\\n\\f]",s="(?:\\\\.|[-\\w]|[^\\x00-\\xa0])+",t=s.replace("w","w#"),u="([*^$|!~]?=)",v="\\["+r+"*("+s+")"+r+"*(?:"+u+r+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+t+")|)|)"+r+"*\\]",w=":("+s+")(?:\\((?:(['\"])((?:\\\\.|[^\\\\])*?)\\2|((?:[^,]|\\\\,|(?:,(?=[^\\[]*\\]))|(?:,(?=[^\\(]*\\))))*))\\)|)",x=":(nth|eq|gt|lt|first|last|even|odd)(?:\\((\\d*)\\)|)(?=[^-]|$)",y=r+"*([\\x20\\t\\r\\n\\f>+~])"+r+"*",z="(?=[^\\x20\\t\\r\\n\\f])(?:\\\\.|"+v+"|"+w.replace(2,7)+"|[^\\\\(),])+",A=new RegExp("^"+r+"+|((?:^|[^\\\\])(?:\\\\.)*)"+r+"+$","g"),B=new RegExp("^"+y),C=new RegExp(z+"?(?="+r+"*,|$)","g"),D=new RegExp("^(?:(?!,)(?:(?:^|,)"+r+"*"+z+")*?|"+r+"*(.*?))(\\)|$)"),E=new RegExp(z.slice(19,-6)+"\\x20\\t\\r\\n\\f>+~])+|"+y,"g"),F=/^(?:#([\w\-]+)|(\w+)|\.([\w\-]+))$/,G=/[\x20\t\r\n\f]*[+~]/,H=/:not\($/,I=/h\d/i,J=/input|select|textarea|button/i,K=/\\(?!\\)/g,L={ID:new RegExp("^#("+s+")"),CLASS:new RegExp("^\\.("+s+")"),NAME:new RegExp("^\\[name=['\"]?("+s+")['\"]?\\]"),TAG:new RegExp("^("+s.replace("[-","[-\\*")+")"),ATTR:new RegExp("^"+v),PSEUDO:new RegExp("^"+w),CHILD:new RegExp("^:(only|nth|last|first)-child(?:\\("+r+"*(even|odd|(([+-]|)(\\d*)n|)"+r+"*(?:([+-]|)"+r+"*(\\d+)|))"+r+"*\\)|)","i"),POS:new RegExp(x,"ig"),needsContext:new RegExp("^"+r+"*[>+~]|"+x,"i")},M={},N=[],O={},P=[],Q=function(a){return a.sizzleFilter=!0,a},R=function(a){return function(b){return b.nodeName.toLowerCase()==="input"&&b.type===a}},S=function(a){return function(b){var c=b.nodeName.toLowerCase();return(c==="input"||c==="button")&&b.type===a}},T=function(a){var b=!1,c=h.createElement("div");try{b=a(c)}catch(d){}return c=null,b},U=T(function(a){a.innerHTML="<select></select>";var b=typeof a.lastChild.getAttribute("multiple");return b!=="boolean"&&b!=="string"}),V=T(function(a){a.id=q+0,a.innerHTML="<a name='"+q+"'></a><div name='"+q+"'></div>",i.insertBefore(a,i.firstChild);var b=h.getElementsByName&&h.getElementsByName(q).length===2+h.getElementsByName(q+0).length;return g=!h.getElementById(q),i.removeChild(a),b}),W=T(function(a){return a.appendChild(h.createComment("")),a.getElementsByTagName("*").length===0}),X=T(function(a){return a.innerHTML="<a href='#'></a>",a.firstChild&&typeof a.firstChild.getAttribute!==j&&a.firstChild.getAttribute("href")==="#"}),Y=T(function(a){return a.innerHTML="<div class='hidden e'></div><div class='hidden'></div>",!a.getElementsByClassName||a.getElementsByClassName("e").length===0?!1:(a.lastChild.className="e",a.getElementsByClassName("e").length!==1)}),Z=function(a,b,c,d){c=c||[],b=b||h;var e,f,g,i,j=b.nodeType;if(j!==1&&j!==9)return[];if(!a||typeof a!="string")return c;g=ba(b);if(!g&&!d)if(e=F.exec(a))if(i=e[1]){if(j===9){f=b.getElementById(i);if(!f||!f.parentNode)return c;if(f.id===i)return c.push(f),c}else if(b.ownerDocument&&(f=b.ownerDocument.getElementById(i))&&bb(b,f)&&f.id===i)return c.push(f),c}else{if(e[2])return o.apply(c,n.call(b.getElementsByTagName(a),0)),c;if((i=e[3])&&Y&&b.getElementsByClassName)return o.apply(c,n.call(b.getElementsByClassName(i),0)),c}return bm(a,b,c,d,g)},$=Z.selectors={cacheLength:50,match:L,order:["ID","TAG"],attrHandle:{},createPseudo:Q,find:{ID:g?function(a,b,c){if(typeof b.getElementById!==j&&!c){var d=b.getElementById(a);return d&&d.parentNode?[d]:[]}}:function(a,c,d){if(typeof c.getElementById!==j&&!d){var e=c.getElementById(a);return e?e.id===a||typeof e.getAttributeNode!==j&&e.getAttributeNode("id").value===a?[e]:b:[]}},TAG:W?function(a,b){if(typeof b.getElementsByTagName!==j)return b.getElementsByTagName(a)}:function(a,b){var c=b.getElementsByTagName(a);if(a==="*"){var d,e=[],f=0;for(;d=c[f];f++)d.nodeType===1&&e.push(d);return e}return c}},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(K,""),a[3]=(a[4]||a[5]||"").replace(K,""),a[2]==="~="&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),a[1]==="nth"?(a[2]||Z.error(a[0]),a[3]=+(a[3]?a[4]+(a[5]||1):2*(a[2]==="even"||a[2]==="odd")),a[4]=+(a[6]+a[7]||a[2]==="odd")):a[2]&&Z.error(a[0]),a},PSEUDO:function(a){var b,c=a[4];return L.CHILD.test(a[0])?null:(c&&(b=D.exec(c))&&b.pop()&&(a[0]=a[0].slice(0,b[0].length-c.length-1),c=b[0].slice(0,-1)),a.splice(2,3,c||a[3]),a)}},filter:{ID:g?function(a){return a=a.replace(K,""),function(b){return b.getAttribute("id")===a}}:function(a){return a=a.replace(K,""),function(b){var c=typeof b.getAttributeNode!==j&&b.getAttributeNode("id");return c&&c.value===a}},TAG:function(a){return a==="*"?function(){return!0}:(a=a.replace(K,"").toLowerCase(),function(b){return b.nodeName&&b.nodeName.toLowerCase()===a})},CLASS:function(a){var b=M[a];return b||(b=M[a]=new RegExp("(^|"+r+")"+a+"("+r+"|$)"),N.push(a),N.length>$.cacheLength&&delete M[N.shift()]),function(a){return b.test(a.className||typeof a.getAttribute!==j&&a.getAttribute("class")||"")}},ATTR:function(a,b,c){return b?function(d){var e=Z.attr(d,a),f=e+"";if(e==null)return b==="!=";switch(b){case"=":return f===c;case"!=":return f!==c;case"^=":return c&&f.indexOf(c)===0;case"*=":return c&&f.indexOf(c)>-1;case"$=":return c&&f.substr(f.length-c.length)===c;case"~=":return(" "+f+" ").indexOf(c)>-1;case"|=":return f===c||f.substr(0,c.length+1)===c+"-"}}:function(b){return Z.attr(b,a)!=null}},CHILD:function(a,b,c,d){if(a==="nth"){var e=m++;return function(a){var b,f,g=0,h=a;if(c===1&&d===0)return!0;b=a.parentNode;if(b&&(b[q]!==e||!a.sizset)){for(h=b.firstChild;h;h=h.nextSibling)if(h.nodeType===1){h.sizset=++g;if(h===a)break}b[q]=e}return f=a.sizset-d,c===0?f===0:f%c===0&&f/c>=0}}return function(b){var c=b;switch(a){case"only":case"first":while(c=c.previousSibling)if(c.nodeType===1)return!1;if(a==="first")return!0;c=b;case"last":while(c=c.nextSibling)if(c.nodeType===1)return!1;return!0}}},PSEUDO:function(a,b,c,d){var e=$.pseudos[a]||$.pseudos[a.toLowerCase()];return e||Z.error("unsupported pseudo: "+a),e.sizzleFilter?e(b,c,d):e}},pseudos:{not:Q(function(a,b,c){var d=bl(a.replace(A,"$1"),b,c);return function(a){return!d(a)}}),enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&!!a.checked||b==="option"&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},parent:function(a){return!$.pseudos.empty(a)},empty:function(a){var b;a=a.firstChild;while(a){if(a.nodeName>"@"||(b=a.nodeType)===3||b===4)return!1;a=a.nextSibling}return!0},contains:Q(function(a){return function(b){return(b.textContent||b.innerText||bc(b)).indexOf(a)>-1}}),has:Q(function(a){return function(b){return Z(a,b).length>0}}),header:function(a){return I.test(a.nodeName)},text:function(a){var b,c;return a.nodeName.toLowerCase()==="input"&&(b=a.type)==="text"&&((c=a.getAttribute("type"))==null||c.toLowerCase()===b)},radio:R("radio"),checkbox:R("checkbox"),file:R("file"),password:R("password"),image:R("image"),submit:S("submit"),reset:S("reset"),button:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&a.type==="button"||b==="button"},input:function(a){return J.test(a.nodeName)},focus:function(a){var b=a.ownerDocument;return a===b.activeElement&&(!b.hasFocus||b.hasFocus())&&(!!a.type||!!a.href)},active:function(a){return a===a.ownerDocument.activeElement}},setFilters:{first:function(a,b,c){return c?a.slice(1):[a[0]]},last:function(a,b,c){var d=a.pop();return c?a:[d]},even:function(a,b,c){var d=[],e=c?1:0,f=a.length;for(;e<f;e=e+2)d.push(a[e]);return d},odd:function(a,b,c){var d=[],e=c?0:1,f=a.length;for(;e<f;e=e+2)d.push(a[e]);return d},lt:function(a,b,c){return c?a.slice(+b):a.slice(0,+b)},gt:function(a,b,c){return c?a.slice(0,+b+1):a.slice(+b+1)},eq:function(a,b,c){var d=a.splice(+b,1);return c?a:d}}};$.setFilters.nth=$.setFilters.eq,$.filters=$.pseudos,X||($.attrHandle={href:function(a){return a.getAttribute("href",2)},type:function(a){return a.getAttribute("type")}}),V&&($.order.push("NAME"),$.find.NAME=function(a,b){if(typeof b.getElementsByName!==j)return b.getElementsByName(a)}),Y&&($.order.splice(1,0,"CLASS"),$.find.CLASS=function(a,b,c){if(typeof b.getElementsByClassName!==j&&!c)return b.getElementsByClassName(a)});try{n.call(i.childNodes,0)[0].nodeType}catch(_){n=function(a){var b,c=[];for(;b=this[a];a++)c.push(b);return c}}var ba=Z.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?b.nodeName!=="HTML":!1},bb=Z.contains=i.compareDocumentPosition?function(a,b){return!!(a.compareDocumentPosition(b)&16)}:i.contains?function(a,b){var c=a.nodeType===9?a.documentElement:a,d=b.parentNode;return a===d||!!(d&&d.nodeType===1&&c.contains&&c.contains(d))}:function(a,b){while(b=b.parentNode)if(b===a)return!0;return!1},bc=Z.getText=function(a){var b,c="",d=0,e=a.nodeType;if(e){if(e===1||e===9||e===11){if(typeof a.textContent=="string")return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=bc(a)}else if(e===3||e===4)return a.nodeValue}else for(;b=a[d];d++)c+=bc(b);return c};Z.attr=function(a,b){var c,d=ba(a);return d||(b=b.toLowerCase()),$.attrHandle[b]?$.attrHandle[b](a):U||d?a.getAttribute(b):(c=a.getAttributeNode(b),c?typeof a[b]=="boolean"?a[b]?b:null:c.specified?c.value:null:null)},Z.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},[0,0].sort(function(){return l=0}),i.compareDocumentPosition?e=function(a,b){return a===b?(k=!0,0):(!a.compareDocumentPosition||!b.compareDocumentPosition?a.compareDocumentPosition:a.compareDocumentPosition(b)&4)?-1:1}:(e=function(a,b){if(a===b)return k=!0,0;if(a.sourceIndex&&b.sourceIndex)return a.sourceIndex-b.sourceIndex;var c,d,e=[],g=[],h=a.parentNode,i=b.parentNode,j=h;if(h===i)return f(a,b);if(!h)return-1;if(!i)return 1;while(j)e.unshift(j),j=j.parentNode;j=i;while(j)g.unshift(j),j=j.parentNode;c=e.length,d=g.length;for(var l=0;l<c&&l<d;l++)if(e[l]!==g[l])return f(e[l],g[l]);return l===c?f(a,g[l],-1):f(e[l],b,1)},f=function(a,b,c){if(a===b)return c;var d=a.nextSibling;while(d){if(d===b)return-1;d=d.nextSibling}return 1}),Z.uniqueSort=function(a){var b,c=1;if(e){k=l,a.sort(e);if(k)for(;b=a[c];c++)b===a[c-1]&&a.splice(c--,1)}return a};var bl=Z.compile=function(a,b,c){var d,e,f,g=O[a];if(g&&g.context===b)return g;e=bg(a,b,c);for(f=0;d=e[f];f++)e[f]=bj(d,b,c);return g=O[a]=bk(e),g.context=b,g.runs=g.dirruns=0,P.push(a),P.length>$.cacheLength&&delete O[P.shift()],g};Z.matches=function(a,b){return Z(a,null,null,b)},Z.matchesSelector=function(a,b){return Z(b,null,null,[a]).length>0};var bm=function(a,b,e,f,g){a=a.replace(A,"$1");var h,i,j,k,l,m,p,q,r,s=a.match(C),t=a.match(E),u=b.nodeType;if(L.POS.test(a))return bf(a,b,e,f,s);if(f)h=n.call(f,0);else if(s&&s.length===1){if(t.length>1&&u===9&&!g&&(s=L.ID.exec(t[0]))){b=$.find.ID(s[1],b,g)[0];if(!b)return e;a=a.slice(t.shift().length)}q=(s=G.exec(t[0]))&&!s.index&&b.parentNode||b,r=t.pop(),m=r.split(":not")[0];for(j=0,k=$.order.length;j<k;j++){p=$.order[j];if(s=L[p].exec(m)){h=$.find[p]((s[1]||"").replace(K,""),q,g);if(h==null)continue;m===r&&(a=a.slice(0,a.length-r.length)+m.replace(L[p],""),a||o.apply(e,n.call(h,0)));break}}}if(a){i=bl(a,b,g),d=i.dirruns++,h==null&&(h=$.find.TAG("*",G.test(a)&&b.parentNode||b));for(j=0;l=h[j];j++)c=i.runs++,i(l,b)&&e.push(l)}return e};h.querySelectorAll&&function(){var a,b=bm,c=/'|\\/g,d=/\=[\x20\t\r\n\f]*([^'"\]]*)[\x20\t\r\n\f]*\]/g,e=[],f=[":active"],g=i.matchesSelector||i.mozMatchesSelector||i.webkitMatchesSelector||i.oMatchesSelector||i.msMatchesSelector;T(function(a){a.innerHTML="<select><option selected></option></select>",a.querySelectorAll("[selected]").length||e.push("\\["+r+"*(?:checked|disabled|ismap|multiple|readonly|selected|value)"),a.querySelectorAll(":checked").length||e.push(":checked")}),T(function(a){a.innerHTML="<p test=''></p>",a.querySelectorAll("[test^='']").length&&e.push("[*^$]="+r+"*(?:\"\"|'')"),a.innerHTML="<input type='hidden'>",a.querySelectorAll(":enabled").length||e.push(":enabled",":disabled")}),e=e.length&&new RegExp(e.join("|")),bm=function(a,d,f,g,h){if(!g&&!h&&(!e||!e.test(a)))if(d.nodeType===9)try{return o.apply(f,n.call(d.querySelectorAll(a),0)),f}catch(i){}else if(d.nodeType===1&&d.nodeName.toLowerCase()!=="object"){var j=d.getAttribute("id"),k=j||q,l=G.test(a)&&d.parentNode||d;j?k=k.replace(c,"\\$&"):d.setAttribute("id",k);try{return o.apply(f,n.call(l.querySelectorAll(a.replace(C,"[id='"+k+"'] $&")),0)),f}catch(i){}finally{j||d.removeAttribute("id")}}return b(a,d,f,g,h)},g&&(T(function(b){a=g.call(b,"div");try{g.call(b,"[test!='']:sizzle"),f.push($.match.PSEUDO)}catch(c){}}),f=new RegExp(f.join("|")),Z.matchesSelector=function(b,c){c=c.replace(d,"='$1']");if(!ba(b)&&!f.test(c)&&(!e||!e.test(c)))try{var h=g.call(b,c);if(h||a||b.document&&b.document.nodeType!==11)return h}catch(i){}return Z(c,null,null,[b]).length>0})}(),Z.attr=p.attr,p.find=Z,p.expr=Z.selectors,p.expr[":"]=p.expr.pseudos,p.unique=Z.uniqueSort,p.text=Z.getText,p.isXMLDoc=Z.isXML,p.contains=Z.contains}(a);var bc=/Until$/,bd=/^(?:parents|prev(?:Until|All))/,be=/^.[^:#\[\.,]*$/,bf=p.expr.match.needsContext,bg={children:!0,contents:!0,next:!0,prev:!0};p.fn.extend({find:function(a){var b,c,d,e,f,g,h=this;if(typeof a!="string")return p(a).filter(function(){for(b=0,c=h.length;b<c;b++)if(p.contains(h[b],this))return!0});g=this.pushStack("","find",a);for(b=0,c=this.length;b<c;b++){d=g.length,p.find(a,this[b],g);if(b>0)for(e=d;e<g.length;e++)for(f=0;f<d;f++)if(g[f]===g[e]){g.splice(e--,1);break}}return g},has:function(a){var b,c=p(a,this),d=c.length;return this.filter(function(){for(b=0;b<d;b++)if(p.contains(this,c[b]))return!0})},not:function(a){return this.pushStack(bj(this,a,!1),"not",a)},filter:function(a){return this.pushStack(bj(this,a,!0),"filter",a)},is:function(a){return!!a&&(typeof a=="string"?bf.test(a)?p(a,this.context).index(this[0])>=0:p.filter(a,this).length>0:this.filter(a).length>0)},closest:function(a,b){var c,d=0,e=this.length,f=[],g=bf.test(a)||typeof a!="string"?p(a,b||this.context):0;for(;d<e;d++){c=this[d];while(c&&c.ownerDocument&&c!==b&&c.nodeType!==11){if(g?g.index(c)>-1:p.find.matchesSelector(c,a)){f.push(c);break}c=c.parentNode}}return f=f.length>1?p.unique(f):f,this.pushStack(f,"closest",a)},index:function(a){return a?typeof a=="string"?p.inArray(this[0],p(a)):p.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.prevAll().length:-1},add:function(a,b){var c=typeof a=="string"?p(a,b):p.makeArray(a&&a.nodeType?[a]:a),d=p.merge(this.get(),c);return this.pushStack(bh(c[0])||bh(d[0])?d:p.unique(d))},addBack:function(a){return this.add(a==null?this.prevObject:this.prevObject.filter(a))}}),p.fn.andSelf=p.fn.addBack,p.each({parent:function(a){var b=a.parentNode;return b&&b.nodeType!==11?b:null},parents:function(a){return p.dir(a,"parentNode")},parentsUntil:function(a,b,c){return p.dir(a,"parentNode",c)},next:function(a){return bi(a,"nextSibling")},prev:function(a){return bi(a,"previousSibling")},nextAll:function(a){return p.dir(a,"nextSibling")},prevAll:function(a){return p.dir(a,"previousSibling")},nextUntil:function(a,b,c){return p.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return p.dir(a,"previousSibling",c)},siblings:function(a){return p.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return p.sibling(a.firstChild)},contents:function(a){return p.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:p.merge([],a.childNodes)}},function(a,b){p.fn[a]=function(c,d){var e=p.map(this,b,c);return bc.test(a)||(d=c),d&&typeof d=="string"&&(e=p.filter(d,e)),e=this.length>1&&!bg[a]?p.unique(e):e,this.length>1&&bd.test(a)&&(e=e.reverse()),this.pushStack(e,a,k.call(arguments).join(","))}}),p.extend({filter:function(a,b,c){return c&&(a=":not("+a+")"),b.length===1?p.find.matchesSelector(b[0],a)?[b[0]]:[]:p.find.matches(a,b)},dir:function(a,c,d){var e=[],f=a[c];while(f&&f.nodeType!==9&&(d===b||f.nodeType!==1||!p(f).is(d)))f.nodeType===1&&e.push(f),f=f[c];return e},sibling:function(a,b){var c=[];for(;a;a=a.nextSibling)a.nodeType===1&&a!==b&&c.push(a);return c}});var bl="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",bm=/ jQuery\d+="(?:null|\d+)"/g,bn=/^\s+/,bo=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bp=/<([\w:]+)/,bq=/<tbody/i,br=/<|&#?\w+;/,bs=/<(?:script|style|link)/i,bt=/<(?:script|object|embed|option|style)/i,bu=new RegExp("<(?:"+bl+")[\\s/>]","i"),bv=/^(?:checkbox|radio)$/,bw=/checked\s*(?:[^=]|=\s*.checked.)/i,bx=/\/(java|ecma)script/i,by=/^\s*<!(?:\[CDATA\[|\-\-)|[\]\-]{2}>\s*$/g,bz={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></table>"],area:[1,"<map>","</map>"],_default:[0,"",""]},bA=bk(e),bB=bA.appendChild(e.createElement("div"));bz.optgroup=bz.option,bz.tbody=bz.tfoot=bz.colgroup=bz.caption=bz.thead,bz.th=bz.td,p.support.htmlSerialize||(bz._default=[1,"X<div>","</div>"]),p.fn.extend({text:function(a){return p.access(this,function(a){return a===b?p.text(this):this.empty().append((this[0]&&this[0].ownerDocument||e).createTextNode(a))},null,a,arguments.length)},wrapAll:function(a){if(p.isFunction(a))return this.each(function(b){p(this).wrapAll(a.call(this,b))});if(this[0]){var b=p(a,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstChild&&a.firstChild.nodeType===1)a=a.firstChild;return a}).append(this)}return this},wrapInner:function(a){return p.isFunction(a)?this.each(function(b){p(this).wrapInner(a.call(this,b))}):this.each(function(){var b=p(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=p.isFunction(a);return this.each(function(c){p(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(){return this.parent().each(function(){p.nodeName(this,"body")||p(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.appendChild(a)})},prepend:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.insertBefore(a,this.firstChild)})},before:function(){if(!bh(this[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(a,this),"before",this.selector)}},after:function(){if(!bh(this[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this.nextSibling)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(this,a),"after",this.selector)}},remove:function(a,b){var c,d=0;for(;(c=this[d])!=null;d++)if(!a||p.filter(a,[c]).length)!b&&c.nodeType===1&&(p.cleanData(c.getElementsByTagName("*")),p.cleanData([c])),c.parentNode&&c.parentNode.removeChild(c);return this},empty:function(){var a,b=0;for(;(a=this[b])!=null;b++){a.nodeType===1&&p.cleanData(a.getElementsByTagName("*"));while(a.firstChild)a.removeChild(a.firstChild)}return this},clone:function(a,b){return a=a==null?!1:a,b=b==null?a:b,this.map(function(){return p.clone(this,a,b)})},html:function(a){return p.access(this,function(a){var c=this[0]||{},d=0,e=this.length;if(a===b)return c.nodeType===1?c.innerHTML.replace(bm,""):b;if(typeof a=="string"&&!bs.test(a)&&(p.support.htmlSerialize||!bu.test(a))&&(p.support.leadingWhitespace||!bn.test(a))&&!bz[(bp.exec(a)||["",""])[1].toLowerCase()]){a=a.replace(bo,"<$1></$2>");try{for(;d<e;d++)c=this[d]||{},c.nodeType===1&&(p.cleanData(c.getElementsByTagName("*")),c.innerHTML=a);c=0}catch(f){}}c&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(a){return bh(this[0])?this.length?this.pushStack(p(p.isFunction(a)?a():a),"replaceWith",a):this:p.isFunction(a)?this.each(function(b){var c=p(this),d=c.html();c.replaceWith(a.call(this,b,d))}):(typeof a!="string"&&(a=p(a).detach()),this.each(function(){var b=this.nextSibling,c=this.parentNode;p(this).remove(),b?p(b).before(a):p(c).append(a)}))},detach:function(a){return this.remove(a,!0)},domManip:function(a,c,d){a=[].concat.apply([],a);var e,f,g,h,i=0,j=a[0],k=[],l=this.length;if(!p.support.checkClone&&l>1&&typeof j=="string"&&bw.test(j))return this.each(function(){p(this).domManip(a,c,d)});if(p.isFunction(j))return this.each(function(e){var f=p(this);a[0]=j.call(this,e,c?f.html():b),f.domManip(a,c,d)});if(this[0]){e=p.buildFragment(a,this,k),g=e.fragment,f=g.firstChild,g.childNodes.length===1&&(g=f);if(f){c=c&&p.nodeName(f,"tr");for(h=e.cacheable||l-1;i<l;i++)d.call(c&&p.nodeName(this[i],"table")?bC(this[i],"tbody"):this[i],i===h?g:p.clone(g,!0,!0))}g=f=null,k.length&&p.each(k,function(a,b){b.src?p.ajax?p.ajax({url:b.src,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0}):p.error("no ajax"):p.globalEval((b.text||b.textContent||b.innerHTML||"").replace(by,"")),b.parentNode&&b.parentNode.removeChild(b)})}return this}}),p.buildFragment=function(a,c,d){var f,g,h,i=a[0];return c=c||e,c=(c[0]||c).ownerDocument||c[0]||c,typeof c.createDocumentFragment=="undefined"&&(c=e),a.length===1&&typeof i=="string"&&i.length<512&&c===e&&i.charAt(0)==="<"&&!bt.test(i)&&(p.support.checkClone||!bw.test(i))&&(p.support.html5Clone||!bu.test(i))&&(g=!0,f=p.fragments[i],h=f!==b),f||(f=c.createDocumentFragment(),p.clean(a,c,f,d),g&&(p.fragments[i]=h&&f)),{fragment:f,cacheable:g}},p.fragments={},p.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){p.fn[a]=function(c){var d,e=0,f=[],g=p(c),h=g.length,i=this.length===1&&this[0].parentNode;if((i==null||i&&i.nodeType===11&&i.childNodes.length===1)&&h===1)return g[b](this[0]),this;for(;e<h;e++)d=(e>0?this.clone(!0):this).get(),p(g[e])[b](d),f=f.concat(d);return this.pushStack(f,a,g.selector)}}),p.extend({clone:function(a,b,c){var d,e,f,g;p.support.html5Clone||p.isXMLDoc(a)||!bu.test("<"+a.nodeName+">")?g=a.cloneNode(!0):(bB.innerHTML=a.outerHTML,bB.removeChild(g=bB.firstChild));if((!p.support.noCloneEvent||!p.support.noCloneChecked)&&(a.nodeType===1||a.nodeType===11)&&!p.isXMLDoc(a)){bE(a,g),d=bF(a),e=bF(g);for(f=0;d[f];++f)e[f]&&bE(d[f],e[f])}if(b){bD(a,g);if(c){d=bF(a),e=bF(g);for(f=0;d[f];++f)bD(d[f],e[f])}}return d=e=null,g},clean:function(a,b,c,d){var f,g,h,i,j,k,l,m,n,o,q,r,s=0,t=[];if(!b||typeof b.createDocumentFragment=="undefined")b=e;for(g=b===e&&bA;(h=a[s])!=null;s++){typeof h=="number"&&(h+="");if(!h)continue;if(typeof h=="string")if(!br.test(h))h=b.createTextNode(h);else{g=g||bk(b),l=l||g.appendChild(b.createElement("div")),h=h.replace(bo,"<$1></$2>"),i=(bp.exec(h)||["",""])[1].toLowerCase(),j=bz[i]||bz._default,k=j[0],l.innerHTML=j[1]+h+j[2];while(k--)l=l.lastChild;if(!p.support.tbody){m=bq.test(h),n=i==="table"&&!m?l.firstChild&&l.firstChild.childNodes:j[1]==="<table>"&&!m?l.childNodes:[];for(f=n.length-1;f>=0;--f)p.nodeName(n[f],"tbody")&&!n[f].childNodes.length&&n[f].parentNode.removeChild(n[f])}!p.support.leadingWhitespace&&bn.test(h)&&l.insertBefore(b.createTextNode(bn.exec(h)[0]),l.firstChild),h=l.childNodes,l=g.lastChild}h.nodeType?t.push(h):t=p.merge(t,h)}l&&(g.removeChild(l),h=l=g=null);if(!p.support.appendChecked)for(s=0;(h=t[s])!=null;s++)p.nodeName(h,"input")?bG(h):typeof h.getElementsByTagName!="undefined"&&p.grep(h.getElementsByTagName("input"),bG);if(c){q=function(a){if(!a.type||bx.test(a.type))return d?d.push(a.parentNode?a.parentNode.removeChild(a):a):c.appendChild(a)};for(s=0;(h=t[s])!=null;s++)if(!p.nodeName(h,"script")||!q(h))c.appendChild(h),typeof h.getElementsByTagName!="undefined"&&(r=p.grep(p.merge([],h.getElementsByTagName("script")),q),t.splice.apply(t,[s+1,0].concat(r)),s+=r.length)}return t},cleanData:function(a,b){var c,d,e,f,g=0,h=p.expando,i=p.cache,j=p.support.deleteExpando,k=p.event.special;for(;(e=a[g])!=null;g++)if(b||p.acceptData(e)){d=e[h],c=d&&i[d];if(c){if(c.events)for(f in c.events)k[f]?p.event.remove(e,f):p.removeEvent(e,f,c.handle);i[d]&&(delete i[d],j?delete e[h]:e.removeAttribute?e.removeAttribute(h):e[h]=null,p.deletedIds.push(d))}}}}),function(){var a,b;p.uaMatch=function(a){a=a.toLowerCase();var b=/(chrome)[ \/]([\w.]+)/.exec(a)||/(webkit)[ \/]([\w.]+)/.exec(a)||/(opera)(?:.*version|)[ \/]([\w.]+)/.exec(a)||/(msie) ([\w.]+)/.exec(a)||a.indexOf("compatible")<0&&/(mozilla)(?:.*? rv:([\w.]+)|)/.exec(a)||[];return{browser:b[1]||"",version:b[2]||"0"}},a=p.uaMatch(g.userAgent),b={},a.browser&&(b[a.browser]=!0,b.version=a.version),b.webkit&&(b.safari=!0),p.browser=b,p.sub=function(){function a(b,c){return new a.fn.init(b,c)}p.extend(!0,a,this),a.superclass=this,a.fn=a.prototype=this(),a.fn.constructor=a,a.sub=this.sub,a.fn.init=function c(c,d){return d&&d instanceof p&&!(d instanceof a)&&(d=a(d)),p.fn.init.call(this,c,d,b)},a.fn.init.prototype=a.fn;var b=a(e);return a}}();var bH,bI,bJ,bK=/alpha\([^)]*\)/i,bL=/opacity=([^)]*)/,bM=/^(top|right|bottom|left)$/,bN=/^margin/,bO=new RegExp("^("+q+")(.*)$","i"),bP=new RegExp("^("+q+")(?!px)[a-z%]+$","i"),bQ=new RegExp("^([-+])=("+q+")","i"),bR={},bS={position:"absolute",visibility:"hidden",display:"block"},bT={letterSpacing:0,fontWeight:400,lineHeight:1},bU=["Top","Right","Bottom","Left"],bV=["Webkit","O","Moz","ms"],bW=p.fn.toggle;p.fn.extend({css:function(a,c){return p.access(this,function(a,c,d){return d!==b?p.style(a,c,d):p.css(a,c)},a,c,arguments.length>1)},show:function(){return bZ(this,!0)},hide:function(){return bZ(this)},toggle:function(a,b){var c=typeof a=="boolean";return p.isFunction(a)&&p.isFunction(b)?bW.apply(this,arguments):this.each(function(){(c?a:bY(this))?p(this).show():p(this).hide()})}}),p.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=bH(a,"opacity");return c===""?"1":c}}}},cssNumber:{fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":p.support.cssFloat?"cssFloat":"styleFloat"},style:function(a,c,d,e){if(!a||a.nodeType===3||a.nodeType===8||!a.style)return;var f,g,h,i=p.camelCase(c),j=a.style;c=p.cssProps[i]||(p.cssProps[i]=bX(j,i)),h=p.cssHooks[c]||p.cssHooks[i];if(d===b)return h&&"get"in h&&(f=h.get(a,!1,e))!==b?f:j[c];g=typeof d,g==="string"&&(f=bQ.exec(d))&&(d=(f[1]+1)*f[2]+parseFloat(p.css(a,c)),g="number");if(d==null||g==="number"&&isNaN(d))return;g==="number"&&!p.cssNumber[i]&&(d+="px");if(!h||!("set"in h)||(d=h.set(a,d,e))!==b)try{j[c]=d}catch(k){}},css:function(a,c,d,e){var f,g,h,i=p.camelCase(c);return c=p.cssProps[i]||(p.cssProps[i]=bX(a.style,i)),h=p.cssHooks[c]||p.cssHooks[i],h&&"get"in h&&(f=h.get(a,!0,e)),f===b&&(f=bH(a,c)),f==="normal"&&c in bT&&(f=bT[c]),d||e!==b?(g=parseFloat(f),d||p.isNumeric(g)?g||0:f):f},swap:function(a,b,c){var d,e,f={};for(e in b)f[e]=a.style[e],a.style[e]=b[e];d=c.call(a);for(e in b)a.style[e]=f[e];return d}}),a.getComputedStyle?bH=function(a,b){var c,d,e,f,g=getComputedStyle(a,null),h=a.style;return g&&(c=g[b],c===""&&!p.contains(a.ownerDocument.documentElement,a)&&(c=p.style(a,b)),bP.test(c)&&bN.test(b)&&(d=h.width,e=h.minWidth,f=h.maxWidth,h.minWidth=h.maxWidth=h.width=c,c=g.width,h.width=d,h.minWidth=e,h.maxWidth=f)),c}:e.documentElement.currentStyle&&(bH=function(a,b){var c,d,e=a.currentStyle&&a.currentStyle[b],f=a.style;return e==null&&f&&f[b]&&(e=f[b]),bP.test(e)&&!bM.test(b)&&(c=f.left,d=a.runtimeStyle&&a.runtimeStyle.left,d&&(a.runtimeStyle.left=a.currentStyle.left),f.left=b==="fontSize"?"1em":e,e=f.pixelLeft+"px",f.left=c,d&&(a.runtimeStyle.left=d)),e===""?"auto":e}),p.each(["height","width"],function(a,b){p.cssHooks[b]={get:function(a,c,d){if(c)return a.offsetWidth!==0||bH(a,"display")!=="none"?ca(a,b,d):p.swap(a,bS,function(){return ca(a,b,d)})},set:function(a,c,d){return b$(a,c,d?b_(a,b,d,p.support.boxSizing&&p.css(a,"boxSizing")==="border-box"):0)}}}),p.support.opacity||(p.cssHooks.opacity={get:function(a,b){return bL.test((b&&a.currentStyle?a.currentStyle.filter:a.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":b?"1":""},set:function(a,b){var c=a.style,d=a.currentStyle,e=p.isNumeric(b)?"alpha(opacity="+b*100+")":"",f=d&&d.filter||c.filter||"";c.zoom=1;if(b>=1&&p.trim(f.replace(bK,""))===""&&c.removeAttribute){c.removeAttribute("filter");if(d&&!d.filter)return}c.filter=bK.test(f)?f.replace(bK,e):f+" "+e}}),p(function(){p.support.reliableMarginRight||(p.cssHooks.marginRight={get:function(a,b){return p.swap(a,{display:"inline-block"},function(){if(b)return bH(a,"marginRight")})}}),!p.support.pixelPosition&&p.fn.position&&p.each(["top","left"],function(a,b){p.cssHooks[b]={get:function(a,c){if(c){var d=bH(a,b);return bP.test(d)?p(a).position()[b]+"px":d}}}})}),p.expr&&p.expr.filters&&(p.expr.filters.hidden=function(a){return a.offsetWidth===0&&a.offsetHeight===0||!p.support.reliableHiddenOffsets&&(a.style&&a.style.display||bH(a,"display"))==="none"},p.expr.filters.visible=function(a){return!p.expr.filters.hidden(a)}),p.each({margin:"",padding:"",border:"Width"},function(a,b){p.cssHooks[a+b]={expand:function(c){var d,e=typeof c=="string"?c.split(" "):[c],f={};for(d=0;d<4;d++)f[a+bU[d]+b]=e[d]||e[d-2]||e[0];return f}},bN.test(a)||(p.cssHooks[a+b].set=b$)});var cc=/%20/g,cd=/\[\]$/,ce=/\r?\n/g,cf=/^(?:color|date|datetime|datetime-local|email|hidden|month|number|password|range|search|tel|text|time|url|week)$/i,cg=/^(?:select|textarea)/i;p.fn.extend({serialize:function(){return p.param(this.serializeArray())},serializeArray:function(){return this.map(function(){return this.elements?p.makeArray(this.elements):this}).filter(function(){return this.name&&!this.disabled&&(this.checked||cg.test(this.nodeName)||cf.test(this.type))}).map(function(a,b){var c=p(this).val();return c==null?null:p.isArray(c)?p.map(c,function(a,c){return{name:b.name,value:a.replace(ce,"\r\n")}}):{name:b.name,value:c.replace(ce,"\r\n")}}).get()}}),p.param=function(a,c){var d,e=[],f=function(a,b){b=p.isFunction(b)?b():b==null?"":b,e[e.length]=encodeURIComponent(a)+"="+encodeURIComponent(b)};c===b&&(c=p.ajaxSettings&&p.ajaxSettings.traditional);if(p.isArray(a)||a.jquery&&!p.isPlainObject(a))p.each(a,function(){f(this.name,this.value)});else for(d in a)ch(d,a[d],c,f);return e.join("&").replace(cc,"+")};var ci,cj,ck=/#.*$/,cl=/^(.*?):[ \t]*([^\r\n]*)\r?$/mg,cm=/^(?:about|app|app\-storage|.+\-extension|file|res|widget):$/,cn=/^(?:GET|HEAD)$/,co=/^\/\//,cp=/\?/,cq=/<script\b[^<]*(?:(?!<\/script>)<[^<]*)*<\/script>/gi,cr=/([?&])_=[^&]*/,cs=/^([\w\+\.\-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,ct=p.fn.load,cu={},cv={},cw=["*/"]+["*"];try{ci=f.href}catch(cx){ci=e.createElement("a"),ci.href="",ci=ci.href}cj=cs.exec(ci.toLowerCase())||[],p.fn.load=function(a,c,d){if(typeof a!="string"&&ct)return ct.apply(this,arguments);if(!this.length)return this;var e,f,g,h=this,i=a.indexOf(" ");return i>=0&&(e=a.slice(i,a.length),a=a.slice(0,i)),p.isFunction(c)?(d=c,c=b):typeof c=="object"&&(f="POST"),p.ajax({url:a,type:f,dataType:"html",data:c,complete:function(a,b){d&&h.each(d,g||[a.responseText,b,a])}}).done(function(a){g=arguments,h.html(e?p("<div>").append(a.replace(cq,"")).find(e):a)}),this},p.each("ajaxStart ajaxStop ajaxComplete ajaxError ajaxSuccess ajaxSend".split(" "),function(a,b){p.fn[b]=function(a){return this.on(b,a)}}),p.each(["get","post"],function(a,c){p[c]=function(a,d,e,f){return p.isFunction(d)&&(f=f||e,e=d,d=b),p.ajax({type:c,url:a,data:d,success:e,dataType:f})}}),p.extend({getScript:function(a,c){return p.get(a,b,c,"script")},getJSON:function(a,b,c){return p.get(a,b,c,"json")},ajaxSetup:function(a,b){return b?cA(a,p.ajaxSettings):(b=a,a=p.ajaxSettings),cA(a,b),a},ajaxSettings:{url:ci,isLocal:cm.test(cj[1]),global:!0,type:"GET",contentType:"application/x-www-form-urlencoded; charset=UTF-8",processData:!0,async:!0,accepts:{xml:"application/xml, text/xml",html:"text/html",text:"text/plain",json:"application/json, text/javascript","*":cw},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":a.String,"text html":!0,"text json":p.parseJSON,"text xml":p.parseXML},flatOptions:{context:!0,url:!0}},ajaxPrefilter:cy(cu),ajaxTransport:cy(cv),ajax:function(a,c){function y(a,c,f,i){var k,s,t,u,w,y=c;if(v===2)return;v=2,h&&clearTimeout(h),g=b,e=i||"",x.readyState=a>0?4:0,f&&(u=cB(l,x,f));if(a>=200&&a<300||a===304)l.ifModified&&(w=x.getResponseHeader("Last-Modified"),w&&(p.lastModified[d]=w),w=x.getResponseHeader("Etag"),w&&(p.etag[d]=w)),a===304?(y="notmodified",k=!0):(k=cC(l,u),y=k.state,s=k.data,t=k.error,k=!t);else{t=y;if(!y||a)y="error",a<0&&(a=0)}x.status=a,x.statusText=""+(c||y),k?o.resolveWith(m,[s,y,x]):o.rejectWith(m,[x,y,t]),x.statusCode(r),r=b,j&&n.trigger("ajax"+(k?"Success":"Error"),[x,l,k?s:t]),q.fireWith(m,[x,y]),j&&(n.trigger("ajaxComplete",[x,l]),--p.active||p.event.trigger("ajaxStop"))}typeof a=="object"&&(c=a,a=b),c=c||{};var d,e,f,g,h,i,j,k,l=p.ajaxSetup({},c),m=l.context||l,n=m!==l&&(m.nodeType||m instanceof p)?p(m):p.event,o=p.Deferred(),q=p.Callbacks("once memory"),r=l.statusCode||{},t={},u={},v=0,w="canceled",x={readyState:0,setRequestHeader:function(a,b){if(!v){var c=a.toLowerCase();a=u[c]=u[c]||a,t[a]=b}return this},getAllResponseHeaders:function(){return v===2?e:null},getResponseHeader:function(a){var c;if(v===2){if(!f){f={};while(c=cl.exec(e))f[c[1].toLowerCase()]=c[2]}c=f[a.toLowerCase()]}return c===b?null:c},overrideMimeType:function(a){return v||(l.mimeType=a),this},abort:function(a){return a=a||w,g&&g.abort(a),y(0,a),this}};o.promise(x),x.success=x.done,x.error=x.fail,x.complete=q.add,x.statusCode=function(a){if(a){var b;if(v<2)for(b in a)r[b]=[r[b],a[b]];else b=a[x.status],x.always(b)}return this},l.url=((a||l.url)+"").replace(ck,"").replace(co,cj[1]+"//"),l.dataTypes=p.trim(l.dataType||"*").toLowerCase().split(s),l.crossDomain==null&&(i=cs.exec(l.url.toLowerCase()),l.crossDomain=!(!i||i[1]==cj[1]&&i[2]==cj[2]&&(i[3]||(i[1]==="http:"?80:443))==(cj[3]||(cj[1]==="http:"?80:443)))),l.data&&l.processData&&typeof l.data!="string"&&(l.data=p.param(l.data,l.traditional)),cz(cu,l,c,x);if(v===2)return x;j=l.global,l.type=l.type.toUpperCase(),l.hasContent=!cn.test(l.type),j&&p.active++===0&&p.event.trigger("ajaxStart");if(!l.hasContent){l.data&&(l.url+=(cp.test(l.url)?"&":"?")+l.data,delete l.data),d=l.url;if(l.cache===!1){var z=p.now(),A=l.url.replace(cr,"$1_="+z);l.url=A+(A===l.url?(cp.test(l.url)?"&":"?")+"_="+z:"")}}(l.data&&l.hasContent&&l.contentType!==!1||c.contentType)&&x.setRequestHeader("Content-Type",l.contentType),l.ifModified&&(d=d||l.url,p.lastModified[d]&&x.setRequestHeader("If-Modified-Since",p.lastModified[d]),p.etag[d]&&x.setRequestHeader("If-None-Match",p.etag[d])),x.setRequestHeader("Accept",l.dataTypes[0]&&l.accepts[l.dataTypes[0]]?l.accepts[l.dataTypes[0]]+(l.dataTypes[0]!=="*"?", "+cw+"; q=0.01":""):l.accepts["*"]);for(k in l.headers)x.setRequestHeader(k,l.headers[k]);if(!l.beforeSend||l.beforeSend.call(m,x,l)!==!1&&v!==2){w="abort";for(k in{success:1,error:1,complete:1})x[k](l[k]);g=cz(cv,l,c,x);if(!g)y(-1,"No Transport");else{x.readyState=1,j&&n.trigger("ajaxSend",[x,l]),l.async&&l.timeout>0&&(h=setTimeout(function(){x.abort("timeout")},l.timeout));try{v=1,g.send(t,y)}catch(B){if(v<2)y(-1,B);else throw B}}return x}return x.abort()},active:0,lastModified:{},etag:{}});var cD=[],cE=/\?/,cF=/(=)\?(?=&|$)|\?\?/,cG=p.now();p.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var a=cD.pop()||p.expando+"_"+cG++;return this[a]=!0,a}}),p.ajaxPrefilter("json jsonp",function(c,d,e){var f,g,h,i=c.data,j=c.url,k=c.jsonp!==!1,l=k&&cF.test(j),m=k&&!l&&typeof i=="string"&&!(c.contentType||"").indexOf("application/x-www-form-urlencoded")&&cF.test(i);if(c.dataTypes[0]==="jsonp"||l||m)return f=c.jsonpCallback=p.isFunction(c.jsonpCallback)?c.jsonpCallback():c.jsonpCallback,g=a[f],l?c.url=j.replace(cF,"$1"+f):m?c.data=i.replace(cF,"$1"+f):k&&(c.url+=(cE.test(j)?"&":"?")+c.jsonp+"="+f),c.converters["script json"]=function(){return h||p.error(f+" was not called"),h[0]},c.dataTypes[0]="json",a[f]=function(){h=arguments},e.always(function(){a[f]=g,c[f]&&(c.jsonpCallback=d.jsonpCallback,cD.push(f)),h&&p.isFunction(g)&&g(h[0]),h=g=b}),"script"}),p.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/javascript|ecmascript/},converters:{"text script":function(a){return p.globalEval(a),a}}}),p.ajaxPrefilter("script",function(a){a.cache===b&&(a.cache=!1),a.crossDomain&&(a.type="GET",a.global=!1)}),p.ajaxTransport("script",function(a){if(a.crossDomain){var c,d=e.head||e.getElementsByTagName("head")[0]||e.documentElement;return{send:function(f,g){c=e.createElement("script"),c.async="async",a.scriptCharset&&(c.charset=a.scriptCharset),c.src=a.url,c.onload=c.onreadystatechange=function(a,e){if(e||!c.readyState||/loaded|complete/.test(c.readyState))c.onload=c.onreadystatechange=null,d&&c.parentNode&&d.removeChild(c),c=b,e||g(200,"success")},d.insertBefore(c,d.firstChild)},abort:function(){c&&c.onload(0,1)}}}});var cH,cI=a.ActiveXObject?function(){for(var a in cH)cH[a](0,1)}:!1,cJ=0;p.ajaxSettings.xhr=a.ActiveXObject?function(){return!this.isLocal&&cK()||cL()}:cK,function(a){p.extend(p.support,{ajax:!!a,cors:!!a&&"withCredentials"in a})}(p.ajaxSettings.xhr()),p.support.ajax&&p.ajaxTransport(function(c){if(!c.crossDomain||p.support.cors){var d;return{send:function(e,f){var g,h,i=c.xhr();c.username?i.open(c.type,c.url,c.async,c.username,c.password):i.open(c.type,c.url,c.async);if(c.xhrFields)for(h in c.xhrFields)i[h]=c.xhrFields[h];c.mimeType&&i.overrideMimeType&&i.overrideMimeType(c.mimeType),!c.crossDomain&&!e["X-Requested-With"]&&(e["X-Requested-With"]="XMLHttpRequest");try{for(h in e)i.setRequestHeader(h,e[h])}catch(j){}i.send(c.hasContent&&c.data||null),d=function(a,e){var h,j,k,l,m;try{if(d&&(e||i.readyState===4)){d=b,g&&(i.onreadystatechange=p.noop,cI&&delete cH[g]);if(e)i.readyState!==4&&i.abort();else{h=i.status,k=i.getAllResponseHeaders(),l={},m=i.responseXML,m&&m.documentElement&&(l.xml=m);try{l.text=i.responseText}catch(a){}try{j=i.statusText}catch(n){j=""}!h&&c.isLocal&&!c.crossDomain?h=l.text?200:404:h===1223&&(h=204)}}}catch(o){e||f(-1,o)}l&&f(h,j,l,k)},c.async?i.readyState===4?setTimeout(d,0):(g=++cJ,cI&&(cH||(cH={},p(a).unload(cI)),cH[g]=d),i.onreadystatechange=d):d()},abort:function(){d&&d(0,1)}}}});var cM,cN,cO=/^(?:toggle|show|hide)$/,cP=new RegExp("^(?:([-+])=|)("+q+")([a-z%]*)$","i"),cQ=/queueHooks$/,cR=[cX],cS={"*":[function(a,b){var c,d,e,f=this.createTween(a,b),g=cP.exec(b),h=f.cur(),i=+h||0,j=1;if(g){c=+g[2],d=g[3]||(p.cssNumber[a]?"":"px");if(d!=="px"&&i){i=p.css(f.elem,a,!0)||c||1;do e=j=j||".5",i=i/j,p.style(f.elem,a,i+d),j=f.cur()/h;while(j!==1&&j!==e)}f.unit=d,f.start=i,f.end=g[1]?i+(g[1]+1)*c:c}return f}]};p.Animation=p.extend(cV,{tweener:function(a,b){p.isFunction(a)?(b=a,a=["*"]):a=a.split(" ");var c,d=0,e=a.length;for(;d<e;d++)c=a[d],cS[c]=cS[c]||[],cS[c].unshift(b)},prefilter:function(a,b){b?cR.unshift(a):cR.push(a)}}),p.Tween=cY,cY.prototype={constructor:cY,init:function(a,b,c,d,e,f){this.elem=a,this.prop=c,this.easing=e||"swing",this.options=b,this.start=this.now=this.cur(),this.end=d,this.unit=f||(p.cssNumber[c]?"":"px")},cur:function(){var a=cY.propHooks[this.prop];return a&&a.get?a.get(this):cY.propHooks._default.get(this)},run:function(a){var b,c=cY.propHooks[this.prop];return this.pos=b=p.easing[this.easing](a,this.options.duration*a,0,1,this.options.duration),this.now=(this.end-this.start)*b+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),c&&c.set?c.set(this):cY.propHooks._default.set(this),this}},cY.prototype.init.prototype=cY.prototype,cY.propHooks={_default:{get:function(a){var b;return a.elem[a.prop]==null||!!a.elem.style&&a.elem.style[a.prop]!=null?(b=p.css(a.elem,a.prop,!1,""),!b||b==="auto"?0:b):a.elem[a.prop]},set:function(a){p.fx.step[a.prop]?p.fx.step[a.prop](a):a.elem.style&&(a.elem.style[p.cssProps[a.prop]]!=null||p.cssHooks[a.prop])?p.style(a.elem,a.prop,a.now+a.unit):a.elem[a.prop]=a.now}}},cY.propHooks.scrollTop=cY.propHooks.scrollLeft={set:function(a){a.elem.nodeType&&a.elem.parentNode&&(a.elem[a.prop]=a.now)}},p.each(["toggle","show","hide"],function(a,b){var c=p.fn[b];p.fn[b]=function(d,e,f){return d==null||typeof d=="boolean"||!a&&p.isFunction(d)&&p.isFunction(e)?c.apply(this,arguments):this.animate(cZ(b,!0),d,e,f)}}),p.fn.extend({fadeTo:function(a,b,c,d){return this.filter(bY).css("opacity",0).show().end().animate({opacity:b},a,c,d)},animate:function(a,b,c,d){var e=p.isEmptyObject(a),f=p.speed(b,c,d),g=function(){var b=cV(this,p.extend({},a),f);e&&b.stop(!0)};return e||f.queue===!1?this.each(g):this.queue(f.queue,g)},stop:function(a,c,d){var e=function(a){var b=a.stop;delete a.stop,b(d)};return typeof a!="string"&&(d=c,c=a,a=b),c&&a!==!1&&this.queue(a||"fx",[]),this.each(function(){var b=!0,c=a!=null&&a+"queueHooks",f=p.timers,g=p._data(this);if(c)g[c]&&g[c].stop&&e(g[c]);else for(c in g)g[c]&&g[c].stop&&cQ.test(c)&&e(g[c]);for(c=f.length;c--;)f[c].elem===this&&(a==null||f[c].queue===a)&&(f[c].anim.stop(d),b=!1,f.splice(c,1));(b||!d)&&p.dequeue(this,a)})}}),p.each({slideDown:cZ("show"),slideUp:cZ("hide"),slideToggle:cZ("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(a,b){p.fn[a]=function(a,c,d){return this.animate(b,a,c,d)}}),p.speed=function(a,b,c){var d=a&&typeof a=="object"?p.extend({},a):{complete:c||!c&&b||p.isFunction(a)&&a,duration:a,easing:c&&b||b&&!p.isFunction(b)&&b};d.duration=p.fx.off?0:typeof d.duration=="number"?d.duration:d.duration in p.fx.speeds?p.fx.speeds[d.duration]:p.fx.speeds._default;if(d.queue==null||d.queue===!0)d.queue="fx";return d.old=d.complete,d.complete=function(){p.isFunction(d.old)&&d.old.call(this),d.queue&&p.dequeue(this,d.queue)},d},p.easing={linear:function(a){return a},swing:function(a){return.5-Math.cos(a*Math.PI)/2}},p.timers=[],p.fx=cY.prototype.init,p.fx.tick=function(){var a,b=p.timers,c=0;for(;c<b.length;c++)a=b[c],!a()&&b[c]===a&&b.splice(c--,1);b.length||p.fx.stop()},p.fx.timer=function(a){a()&&p.timers.push(a)&&!cN&&(cN=setInterval(p.fx.tick,p.fx.interval))},p.fx.interval=13,p.fx.stop=function(){clearInterval(cN),cN=null},p.fx.speeds={slow:600,fast:200,_default:400},p.fx.step={},p.expr&&p.expr.filters&&(p.expr.filters.animated=function(a){return p.grep(p.timers,function(b){return a===b.elem}).length});var c$=/^(?:body|html)$/i;p.fn.offset=function(a){if(arguments.length)return a===b?this:this.each(function(b){p.offset.setOffset(this,a,b)});var c,d,e,f,g,h,i,j,k,l,m=this[0],n=m&&m.ownerDocument;if(!n)return;return(e=n.body)===m?p.offset.bodyOffset(m):(d=n.documentElement,p.contains(d,m)?(c=m.getBoundingClientRect(),f=c_(n),g=d.clientTop||e.clientTop||0,h=d.clientLeft||e.clientLeft||0,i=f.pageYOffset||d.scrollTop,j=f.pageXOffset||d.scrollLeft,k=c.top+i-g,l=c.left+j-h,{top:k,left:l}):{top:0,left:0})},p.offset={bodyOffset:function(a){var b=a.offsetTop,c=a.offsetLeft;return p.support.doesNotIncludeMarginInBodyOffset&&(b+=parseFloat(p.css(a,"marginTop"))||0,c+=parseFloat(p.css(a,"marginLeft"))||0),{top:b,left:c}},setOffset:function(a,b,c){var d=p.css(a,"position");d==="static"&&(a.style.position="relative");var e=p(a),f=e.offset(),g=p.css(a,"top"),h=p.css(a,"left"),i=(d==="absolute"||d==="fixed")&&p.inArray("auto",[g,h])>-1,j={},k={},l,m;i?(k=e.position(),l=k.top,m=k.left):(l=parseFloat(g)||0,m=parseFloat(h)||0),p.isFunction(b)&&(b=b.call(a,c,f)),b.top!=null&&(j.top=b.top-f.top+l),b.left!=null&&(j.left=b.left-f.left+m),"using"in b?b.using.call(a,j):e.css(j)}},p.fn.extend({position:function(){if(!this[0])return;var a=this[0],b=this.offsetParent(),c=this.offset(),d=c$.test(b[0].nodeName)?{top:0,left:0}:b.offset();return c.top-=parseFloat(p.css(a,"marginTop"))||0,c.left-=parseFloat(p.css(a,"marginLeft"))||0,d.top+=parseFloat(p.css(b[0],"borderTopWidth"))||0,d.left+=parseFloat(p.css(b[0],"borderLeftWidth"))||0,{top:c.top-d.top,left:c.left-d.left}},offsetParent:function(){return this.map(function(){var a=this.offsetParent||e.body;while(a&&!c$.test(a.nodeName)&&p.css(a,"position")==="static")a=a.offsetParent;return a||e.body})}}),p.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(a,c){var d=/Y/.test(c);p.fn[a]=function(e){return p.access(this,function(a,e,f){var g=c_(a);if(f===b)return g?c in g?g[c]:g.document.documentElement[e]:a[e];g?g.scrollTo(d?p(g).scrollLeft():f,d?f:p(g).scrollTop()):a[e]=f},a,e,arguments.length,null)}}),p.each({Height:"height",Width:"width"},function(a,c){p.each({padding:"inner"+a,content:c,"":"outer"+a},function(d,e){p.fn[e]=function(e,f){var g=arguments.length&&(d||typeof e!="boolean"),h=d||(e===!0||f===!0?"margin":"border");return p.access(this,function(c,d,e){var f;return p.isWindow(c)?c.document.documentElement["client"+a]:c.nodeType===9?(f=c.documentElement,Math.max(c.body["scroll"+a],f["scroll"+a],c.body["offset"+a],f["offset"+a],f["client"+a])):e===b?p.css(c,d,e,h):p.style(c,d,e,h)},c,g?e:b,g)}})}),a.jQuery=a.$=p,typeof define=="function"&&define.amd&&define.amd.jQuery&&define("jquery",[],function(){return p})})(window); \ No newline at end of file
diff --git a/docs/js/vendor/modernizr-2.6.1-respond-1.1.0.min.js b/docs/js/vendor/modernizr-2.6.1-respond-1.1.0.min.js
new file mode 100755
index 0000000000..8e708949c0
--- /dev/null
+++ b/docs/js/vendor/modernizr-2.6.1-respond-1.1.0.min.js
@@ -0,0 +1,11 @@
+/* Modernizr 2.6.1 (Custom Build) | MIT & BSD
+ * Build: http://modernizr.com/download/#-fontface-backgroundsize-borderimage-borderradius-boxshadow-flexbox-hsla-multiplebgs-opacity-rgba-textshadow-cssanimations-csscolumns-generatedcontent-cssgradients-cssreflections-csstransforms-csstransforms3d-csstransitions-applicationcache-canvas-canvastext-draganddrop-hashchange-history-audio-video-indexeddb-input-inputtypes-localstorage-postmessage-sessionstorage-websockets-websqldatabase-webworkers-geolocation-inlinesvg-smil-svg-svgclippaths-touch-webgl-shiv-mq-cssclasses-addtest-prefixed-teststyles-testprop-testallprops-hasevent-prefixes-domprefixes-load
+ */
+;window.Modernizr=function(a,b,c){function D(a){j.cssText=a}function E(a,b){return D(n.join(a+";")+(b||""))}function F(a,b){return typeof a===b}function G(a,b){return!!~(""+a).indexOf(b)}function H(a,b){for(var d in a){var e=a[d];if(!G(e,"-")&&j[e]!==c)return b=="pfx"?e:!0}return!1}function I(a,b,d){for(var e in a){var f=b[a[e]];if(f!==c)return d===!1?a[e]:F(f,"function")?f.bind(d||b):f}return!1}function J(a,b,c){var d=a.charAt(0).toUpperCase()+a.slice(1),e=(a+" "+p.join(d+" ")+d).split(" ");return F(b,"string")||F(b,"undefined")?H(e,b):(e=(a+" "+q.join(d+" ")+d).split(" "),I(e,b,c))}function K(){e.input=function(c){for(var d=0,e=c.length;d<e;d++)u[c[d]]=c[d]in k;return u.list&&(u.list=!!b.createElement("datalist")&&!!a.HTMLDataListElement),u}("autocomplete autofocus list placeholder max min multiple pattern required step".split(" ")),e.inputtypes=function(a){for(var d=0,e,f,h,i=a.length;d<i;d++)k.setAttribute("type",f=a[d]),e=k.type!=="text",e&&(k.value=l,k.style.cssText="position:absolute;visibility:hidden;",/^range$/.test(f)&&k.style.WebkitAppearance!==c?(g.appendChild(k),h=b.defaultView,e=h.getComputedStyle&&h.getComputedStyle(k,null).WebkitAppearance!=="textfield"&&k.offsetHeight!==0,g.removeChild(k)):/^(search|tel)$/.test(f)||(/^(url|email)$/.test(f)?e=k.checkValidity&&k.checkValidity()===!1:e=k.value!=l)),t[a[d]]=!!e;return t}("search tel url email datetime date month week time datetime-local number range color".split(" "))}var d="2.6.1",e={},f=!0,g=b.documentElement,h="modernizr",i=b.createElement(h),j=i.style,k=b.createElement("input"),l=":)",m={}.toString,n=" -webkit- -moz- -o- -ms- ".split(" "),o="Webkit Moz O ms",p=o.split(" "),q=o.toLowerCase().split(" "),r={svg:"http://www.w3.org/2000/svg"},s={},t={},u={},v=[],w=v.slice,x,y=function(a,c,d,e){var f,i,j,k=b.createElement("div"),l=b.body,m=l?l:b.createElement("body");if(parseInt(d,10))while(d--)j=b.createElement("div"),j.id=e?e[d]:h+(d+1),k.appendChild(j);return f=["&#173;",'<style id="s',h,'">',a,"</style>"].join(""),k.id=h,(l?k:m).innerHTML+=f,m.appendChild(k),l||(m.style.background="",g.appendChild(m)),i=c(k,a),l?k.parentNode.removeChild(k):m.parentNode.removeChild(m),!!i},z=function(b){var c=a.matchMedia||a.msMatchMedia;if(c)return c(b).matches;var d;return y("@media "+b+" { #"+h+" { position: absolute; } }",function(b){d=(a.getComputedStyle?getComputedStyle(b,null):b.currentStyle)["position"]=="absolute"}),d},A=function(){function d(d,e){e=e||b.createElement(a[d]||"div"),d="on"+d;var f=d in e;return f||(e.setAttribute||(e=b.createElement("div")),e.setAttribute&&e.removeAttribute&&(e.setAttribute(d,""),f=F(e[d],"function"),F(e[d],"undefined")||(e[d]=c),e.removeAttribute(d))),e=null,f}var a={select:"input",change:"input",submit:"form",reset:"form",error:"img",load:"img",abort:"img"};return d}(),B={}.hasOwnProperty,C;!F(B,"undefined")&&!F(B.call,"undefined")?C=function(a,b){return B.call(a,b)}:C=function(a,b){return b in a&&F(a.constructor.prototype[b],"undefined")},Function.prototype.bind||(Function.prototype.bind=function(b){var c=this;if(typeof c!="function")throw new TypeError;var d=w.call(arguments,1),e=function(){if(this instanceof e){var a=function(){};a.prototype=c.prototype;var f=new a,g=c.apply(f,d.concat(w.call(arguments)));return Object(g)===g?g:f}return c.apply(b,d.concat(w.call(arguments)))};return e}),s.flexbox=function(){return J("flexWrap")},s.canvas=function(){var a=b.createElement("canvas");return!!a.getContext&&!!a.getContext("2d")},s.canvastext=function(){return!!e.canvas&&!!F(b.createElement("canvas").getContext("2d").fillText,"function")},s.webgl=function(){return!!a.WebGLRenderingContext},s.touch=function(){var c;return"ontouchstart"in a||a.DocumentTouch&&b instanceof DocumentTouch?c=!0:y(["@media (",n.join("touch-enabled),("),h,")","{#modernizr{top:9px;position:absolute}}"].join(""),function(a){c=a.offsetTop===9}),c},s.geolocation=function(){return"geolocation"in navigator},s.postmessage=function(){return!!a.postMessage},s.websqldatabase=function(){return!!a.openDatabase},s.indexedDB=function(){return!!J("indexedDB",a)},s.hashchange=function(){return A("hashchange",a)&&(b.documentMode===c||b.documentMode>7)},s.history=function(){return!!a.history&&!!history.pushState},s.draganddrop=function(){var a=b.createElement("div");return"draggable"in a||"ondragstart"in a&&"ondrop"in a},s.websockets=function(){return"WebSocket"in a||"MozWebSocket"in a},s.rgba=function(){return D("background-color:rgba(150,255,150,.5)"),G(j.backgroundColor,"rgba")},s.hsla=function(){return D("background-color:hsla(120,40%,100%,.5)"),G(j.backgroundColor,"rgba")||G(j.backgroundColor,"hsla")},s.multiplebgs=function(){return D("background:url(https://),url(https://),red url(https://)"),/(url\s*\(.*?){3}/.test(j.background)},s.backgroundsize=function(){return J("backgroundSize")},s.borderimage=function(){return J("borderImage")},s.borderradius=function(){return J("borderRadius")},s.boxshadow=function(){return J("boxShadow")},s.textshadow=function(){return b.createElement("div").style.textShadow===""},s.opacity=function(){return E("opacity:.55"),/^0.55$/.test(j.opacity)},s.cssanimations=function(){return J("animationName")},s.csscolumns=function(){return J("columnCount")},s.cssgradients=function(){var a="background-image:",b="gradient(linear,left top,right bottom,from(#9f9),to(white));",c="linear-gradient(left top,#9f9, white);";return D((a+"-webkit- ".split(" ").join(b+a)+n.join(c+a)).slice(0,-a.length)),G(j.backgroundImage,"gradient")},s.cssreflections=function(){return J("boxReflect")},s.csstransforms=function(){return!!J("transform")},s.csstransforms3d=function(){var a=!!J("perspective");return a&&"webkitPerspective"in g.style&&y("@media (transform-3d),(-webkit-transform-3d){#modernizr{left:9px;position:absolute;height:3px;}}",function(b,c){a=b.offsetLeft===9&&b.offsetHeight===3}),a},s.csstransitions=function(){return J("transition")},s.fontface=function(){var a;return y('@font-face {font-family:"font";src:url("https://")}',function(c,d){var e=b.getElementById("smodernizr"),f=e.sheet||e.styleSheet,g=f?f.cssRules&&f.cssRules[0]?f.cssRules[0].cssText:f.cssText||"":"";a=/src/i.test(g)&&g.indexOf(d.split(" ")[0])===0}),a},s.generatedcontent=function(){var a;return y(['#modernizr:after{content:"',l,'";visibility:hidden}'].join(""),function(b){a=b.offsetHeight>=1}),a},s.video=function(){var a=b.createElement("video"),c=!1;try{if(c=!!a.canPlayType)c=new Boolean(c),c.ogg=a.canPlayType('video/ogg; codecs="theora"').replace(/^no$/,""),c.h264=a.canPlayType('video/mp4; codecs="avc1.42E01E"').replace(/^no$/,""),c.webm=a.canPlayType('video/webm; codecs="vp8, vorbis"').replace(/^no$/,"")}catch(d){}return c},s.audio=function(){var a=b.createElement("audio"),c=!1;try{if(c=!!a.canPlayType)c=new Boolean(c),c.ogg=a.canPlayType('audio/ogg; codecs="vorbis"').replace(/^no$/,""),c.mp3=a.canPlayType("audio/mpeg;").replace(/^no$/,""),c.wav=a.canPlayType('audio/wav; codecs="1"').replace(/^no$/,""),c.m4a=(a.canPlayType("audio/x-m4a;")||a.canPlayType("audio/aac;")).replace(/^no$/,"")}catch(d){}return c},s.localstorage=function(){try{return localStorage.setItem(h,h),localStorage.removeItem(h),!0}catch(a){return!1}},s.sessionstorage=function(){try{return sessionStorage.setItem(h,h),sessionStorage.removeItem(h),!0}catch(a){return!1}},s.webworkers=function(){return!!a.Worker},s.applicationcache=function(){return!!a.applicationCache},s.svg=function(){return!!b.createElementNS&&!!b.createElementNS(r.svg,"svg").createSVGRect},s.inlinesvg=function(){var a=b.createElement("div");return a.innerHTML="<svg/>",(a.firstChild&&a.firstChild.namespaceURI)==r.svg},s.smil=function(){return!!b.createElementNS&&/SVGAnimate/.test(m.call(b.createElementNS(r.svg,"animate")))},s.svgclippaths=function(){return!!b.createElementNS&&/SVGClipPath/.test(m.call(b.createElementNS(r.svg,"clipPath")))};for(var L in s)C(s,L)&&(x=L.toLowerCase(),e[x]=s[L](),v.push((e[x]?"":"no-")+x));return e.input||K(),e.addTest=function(a,b){if(typeof a=="object")for(var d in a)C(a,d)&&e.addTest(d,a[d]);else{a=a.toLowerCase();if(e[a]!==c)return e;b=typeof b=="function"?b():b,f&&(g.className+=" "+(b?"":"no-")+a),e[a]=b}return e},D(""),i=k=null,function(a,b){function k(a,b){var c=a.createElement("p"),d=a.getElementsByTagName("head")[0]||a.documentElement;return c.innerHTML="x<style>"+b+"</style>",d.insertBefore(c.lastChild,d.firstChild)}function l(){var a=r.elements;return typeof a=="string"?a.split(" "):a}function m(a){var b=i[a[g]];return b||(b={},h++,a[g]=h,i[h]=b),b}function n(a,c,f){c||(c=b);if(j)return c.createElement(a);f||(f=m(c));var g;return f.cache[a]?g=f.cache[a].cloneNode():e.test(a)?g=(f.cache[a]=f.createElem(a)).cloneNode():g=f.createElem(a),g.canHaveChildren&&!d.test(a)?f.frag.appendChild(g):g}function o(a,c){a||(a=b);if(j)return a.createDocumentFragment();c=c||m(a);var d=c.frag.cloneNode(),e=0,f=l(),g=f.length;for(;e<g;e++)d.createElement(f[e]);return d}function p(a,b){b.cache||(b.cache={},b.createElem=a.createElement,b.createFrag=a.createDocumentFragment,b.frag=b.createFrag()),a.createElement=function(c){return r.shivMethods?n(c,a,b):b.createElem(c)},a.createDocumentFragment=Function("h,f","return function(){var n=f.cloneNode(),c=n.createElement;h.shivMethods&&("+l().join().replace(/\w+/g,function(a){return b.createElem(a),b.frag.createElement(a),'c("'+a+'")'})+");return n}")(r,b.frag)}function q(a){a||(a=b);var c=m(a);return r.shivCSS&&!f&&!c.hasCSS&&(c.hasCSS=!!k(a,"article,aside,figcaption,figure,footer,header,hgroup,nav,section{display:block}mark{background:#FF0;color:#000}")),j||p(a,c),a}var c=a.html5||{},d=/^<|^(?:button|map|select|textarea|object|iframe|option|optgroup)$/i,e=/^<|^(?:a|b|button|code|div|fieldset|form|h1|h2|h3|h4|h5|h6|i|iframe|img|input|label|li|link|ol|option|p|param|q|script|select|span|strong|style|table|tbody|td|textarea|tfoot|th|thead|tr|ul)$/i,f,g="_html5shiv",h=0,i={},j;(function(){try{var a=b.createElement("a");a.innerHTML="<xyz></xyz>",f="hidden"in a,j=a.childNodes.length==1||function(){b.createElement("a");var a=b.createDocumentFragment();return typeof a.cloneNode=="undefined"||typeof a.createDocumentFragment=="undefined"||typeof a.createElement=="undefined"}()}catch(c){f=!0,j=!0}})();var r={elements:c.elements||"abbr article aside audio bdi canvas data datalist details figcaption figure footer header hgroup mark meter nav output progress section summary time video",shivCSS:c.shivCSS!==!1,supportsUnknownElements:j,shivMethods:c.shivMethods!==!1,type:"default",shivDocument:q,createElement:n,createDocumentFragment:o};a.html5=r,q(b)}(this,b),e._version=d,e._prefixes=n,e._domPrefixes=q,e._cssomPrefixes=p,e.mq=z,e.hasEvent=A,e.testProp=function(a){return H([a])},e.testAllProps=J,e.testStyles=y,e.prefixed=function(a,b,c){return b?J(a,b,c):J(a,"pfx")},g.className=g.className.replace(/(^|\s)no-js(\s|$)/,"$1$2")+(f?" js "+v.join(" "):""),e}(this,this.document),function(a,b,c){function d(a){return o.call(a)=="[object Function]"}function e(a){return typeof a=="string"}function f(){}function g(a){return!a||a=="loaded"||a=="complete"||a=="uninitialized"}function h(){var a=p.shift();q=1,a?a.t?m(function(){(a.t=="c"?B.injectCss:B.injectJs)(a.s,0,a.a,a.x,a.e,1)},0):(a(),h()):q=0}function i(a,c,d,e,f,i,j){function k(b){if(!o&&g(l.readyState)&&(u.r=o=1,!q&&h(),l.onload=l.onreadystatechange=null,b)){a!="img"&&m(function(){t.removeChild(l)},50);for(var d in y[c])y[c].hasOwnProperty(d)&&y[c][d].onload()}}var j=j||B.errorTimeout,l={},o=0,r=0,u={t:d,s:c,e:f,a:i,x:j};y[c]===1&&(r=1,y[c]=[],l=b.createElement(a)),a=="object"?l.data=c:(l.src=c,l.type=a),l.width=l.height="0",l.onerror=l.onload=l.onreadystatechange=function(){k.call(this,r)},p.splice(e,0,u),a!="img"&&(r||y[c]===2?(t.insertBefore(l,s?null:n),m(k,j)):y[c].push(l))}function j(a,b,c,d,f){return q=0,b=b||"j",e(a)?i(b=="c"?v:u,a,b,this.i++,c,d,f):(p.splice(this.i++,0,a),p.length==1&&h()),this}function k(){var a=B;return a.loader={load:j,i:0},a}var l=b.documentElement,m=a.setTimeout,n=b.getElementsByTagName("script")[0],o={}.toString,p=[],q=0,r="MozAppearance"in l.style,s=r&&!!b.createRange().compareNode,t=s?l:n.parentNode,l=a.opera&&o.call(a.opera)=="[object Opera]",l=!!b.attachEvent&&!l,u=r?"object":l?"script":"img",v=l?"script":u,w=Array.isArray||function(a){return o.call(a)=="[object Array]"},x=[],y={},z={timeout:function(a,b){return b.length&&(a.timeout=b[0]),a}},A,B;B=function(a){function b(a){var a=a.split("!"),b=x.length,c=a.pop(),d=a.length,c={url:c,origUrl:c,prefixes:a},e,f,g;for(f=0;f<d;f++)g=a[f].split("="),(e=z[g.shift()])&&(c=e(c,g));for(f=0;f<b;f++)c=x[f](c);return c}function g(a,e,f,g,i){var j=b(a),l=j.autoCallback;j.url.split(".").pop().split("?").shift(),j.bypass||(e&&(e=d(e)?e:e[a]||e[g]||e[a.split("/").pop().split("?")[0]]||h),j.instead?j.instead(a,e,f,g,i):(y[j.url]?j.noexec=!0:y[j.url]=1,f.load(j.url,j.forceCSS||!j.forceJS&&"css"==j.url.split(".").pop().split("?").shift()?"c":c,j.noexec,j.attrs,j.timeout),(d(e)||d(l))&&f.load(function(){k(),e&&e(j.origUrl,i,g),l&&l(j.origUrl,i,g),y[j.url]=2})))}function i(a,b){function c(a,c){if(a){if(e(a))c||(j=function(){var a=[].slice.call(arguments);k.apply(this,a),l()}),g(a,j,b,0,h);else if(Object(a)===a)for(n in m=function(){var b=0,c;for(c in a)a.hasOwnProperty(c)&&b++;return b}(),a)a.hasOwnProperty(n)&&(!c&&!--m&&(d(j)?j=function(){var a=[].slice.call(arguments);k.apply(this,a),l()}:j[n]=function(a){return function(){var b=[].slice.call(arguments);a&&a.apply(this,b),l()}}(k[n])),g(a[n],j,b,n,h))}else!c&&l()}var h=!!a.test,i=a.load||a.both,j=a.callback||f,k=j,l=a.complete||f,m,n;c(h?a.yep:a.nope,!!i),i&&c(i)}var j,l,m=this.yepnope.loader;if(e(a))g(a,0,m,0);else if(w(a))for(j=0;j<a.length;j++)l=a[j],e(l)?g(l,0,m,0):w(l)?B(l):Object(l)===l&&i(l,m);else Object(a)===a&&i(a,m)},B.addPrefix=function(a,b){z[a]=b},B.addFilter=function(a){x.push(a)},B.errorTimeout=1e4,b.readyState==null&&b.addEventListener&&(b.readyState="loading",b.addEventListener("DOMContentLoaded",A=function(){b.removeEventListener("DOMContentLoaded",A,0),b.readyState="complete"},0)),a.yepnope=k(),a.yepnope.executeStack=h,a.yepnope.injectJs=function(a,c,d,e,i,j){var k=b.createElement("script"),l,o,e=e||B.errorTimeout;k.src=a;for(o in d)k.setAttribute(o,d[o]);c=j?h:c||f,k.onreadystatechange=k.onload=function(){!l&&g(k.readyState)&&(l=1,c(),k.onload=k.onreadystatechange=null)},m(function(){l||(l=1,c(1))},e),i?k.onload():n.parentNode.insertBefore(k,n)},a.yepnope.injectCss=function(a,c,d,e,g,i){var e=b.createElement("link"),j,c=i?h:c||f;e.href=a,e.rel="stylesheet",e.type="text/css";for(j in d)e.setAttribute(j,d[j]);g||(n.parentNode.insertBefore(e,n),m(c,0))}}(this,document),Modernizr.load=function(){yepnope.apply(window,[].slice.call(arguments,0))};
+
+/*! matchMedia() polyfill - Test a CSS media type/query in JS. Authors & copyright (c) 2012: Scott Jehl, Paul Irish, Nicholas Zakas. Dual MIT/BSD license */
+/*! NOTE: If you're already including a window.matchMedia polyfill via Modernizr or otherwise, you don't need this part */
+window.matchMedia=window.matchMedia||(function(e,f){var c,a=e.documentElement,b=a.firstElementChild||a.firstChild,d=e.createElement("body"),g=e.createElement("div");g.id="mq-test-1";g.style.cssText="position:absolute;top:-100em";d.style.background="none";d.appendChild(g);return function(h){g.innerHTML='&shy;<style media="'+h+'"> #mq-test-1 { width: 42px; }</style>';a.insertBefore(d,b);c=g.offsetWidth==42;a.removeChild(d);return{matches:c,media:h}}})(document);
+
+/*! Respond.js v1.1.0: min/max-width media query polyfill. (c) Scott Jehl. MIT/GPLv2 Lic. j.mp/respondjs */
+(function(e){e.respond={};respond.update=function(){};respond.mediaQueriesSupported=e.matchMedia&&e.matchMedia("only all").matches;if(respond.mediaQueriesSupported){return}var w=e.document,s=w.documentElement,i=[],k=[],q=[],o={},h=30,f=w.getElementsByTagName("head")[0]||s,g=w.getElementsByTagName("base")[0],b=f.getElementsByTagName("link"),d=[],a=function(){var D=b,y=D.length,B=0,A,z,C,x;for(;B<y;B++){A=D[B],z=A.href,C=A.media,x=A.rel&&A.rel.toLowerCase()==="stylesheet";if(!!z&&x&&!o[z]){if(A.styleSheet&&A.styleSheet.rawCssText){m(A.styleSheet.rawCssText,z,C);o[z]=true}else{if((!/^([a-zA-Z:]*\/\/)/.test(z)&&!g)||z.replace(RegExp.$1,"").split("/")[0]===e.location.host){d.push({href:z,media:C})}}}}u()},u=function(){if(d.length){var x=d.shift();n(x.href,function(y){m(y,x.href,x.media);o[x.href]=true;u()})}},m=function(I,x,z){var G=I.match(/@media[^\{]+\{([^\{\}]*\{[^\}\{]*\})+/gi),J=G&&G.length||0,x=x.substring(0,x.lastIndexOf("/")),y=function(K){return K.replace(/(url\()['"]?([^\/\)'"][^:\)'"]+)['"]?(\))/g,"$1"+x+"$2$3")},A=!J&&z,D=0,C,E,F,B,H;if(x.length){x+="/"}if(A){J=1}for(;D<J;D++){C=0;if(A){E=z;k.push(y(I))}else{E=G[D].match(/@media *([^\{]+)\{([\S\s]+?)$/)&&RegExp.$1;k.push(RegExp.$2&&y(RegExp.$2))}B=E.split(",");H=B.length;for(;C<H;C++){F=B[C];i.push({media:F.split("(")[0].match(/(only\s+)?([a-zA-Z]+)\s?/)&&RegExp.$2||"all",rules:k.length-1,hasquery:F.indexOf("(")>-1,minw:F.match(/\(min\-width:[\s]*([\s]*[0-9\.]+)(px|em)[\s]*\)/)&&parseFloat(RegExp.$1)+(RegExp.$2||""),maxw:F.match(/\(max\-width:[\s]*([\s]*[0-9\.]+)(px|em)[\s]*\)/)&&parseFloat(RegExp.$1)+(RegExp.$2||"")})}}j()},l,r,v=function(){var z,A=w.createElement("div"),x=w.body,y=false;A.style.cssText="position:absolute;font-size:1em;width:1em";if(!x){x=y=w.createElement("body");x.style.background="none"}x.appendChild(A);s.insertBefore(x,s.firstChild);z=A.offsetWidth;if(y){s.removeChild(x)}else{x.removeChild(A)}z=p=parseFloat(z);return z},p,j=function(I){var x="clientWidth",B=s[x],H=w.compatMode==="CSS1Compat"&&B||w.body[x]||B,D={},G=b[b.length-1],z=(new Date()).getTime();if(I&&l&&z-l<h){clearTimeout(r);r=setTimeout(j,h);return}else{l=z}for(var E in i){var K=i[E],C=K.minw,J=K.maxw,A=C===null,L=J===null,y="em";if(!!C){C=parseFloat(C)*(C.indexOf(y)>-1?(p||v()):1)}if(!!J){J=parseFloat(J)*(J.indexOf(y)>-1?(p||v()):1)}if(!K.hasquery||(!A||!L)&&(A||H>=C)&&(L||H<=J)){if(!D[K.media]){D[K.media]=[]}D[K.media].push(k[K.rules])}}for(var E in q){if(q[E]&&q[E].parentNode===f){f.removeChild(q[E])}}for(var E in D){var M=w.createElement("style"),F=D[E].join("\n");M.type="text/css";M.media=E;f.insertBefore(M,G.nextSibling);if(M.styleSheet){M.styleSheet.cssText=F}else{M.appendChild(w.createTextNode(F))}q.push(M)}},n=function(x,z){var y=c();if(!y){return}y.open("GET",x,true);y.onreadystatechange=function(){if(y.readyState!=4||y.status!=200&&y.status!=304){return}z(y.responseText)};if(y.readyState==4){return}y.send(null)},c=(function(){var x=false;try{x=new XMLHttpRequest()}catch(y){x=new ActiveXObject("Microsoft.XMLHTTP")}return function(){return x}})();a();respond.update=a;function t(){j(true)}if(e.addEventListener){e.addEventListener("resize",t,false)}else{if(e.attachEvent){e.attachEvent("onresize",t)}}})(this); \ No newline at end of file
diff --git a/docs/quick-start.md b/docs/quick-start.md
new file mode 100644
index 0000000000..defdb34836
--- /dev/null
+++ b/docs/quick-start.md
@@ -0,0 +1,232 @@
+---
+layout: global
+title: Quick Start
+---
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will need much for this), then show how to write standalone jobs in Scala and Java. See the [programming guide](scala-programming-guide.html) for a fuller reference.
+
+To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run:
+
+{% highlight bash %}
+$ sbt/sbt package
+{% endhighlight %}
+
+# Interactive Analysis with the Spark Shell
+
+## 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.
+
+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:
+
+{% highlight scala %}
+scala> val textFile = sc.textFile("README.md")
+textFile: spark.RDD[String] = spark.MappedRDD@2ee9b6e3
+{% endhighlight %}
+
+RDDs have _[actions](scala-programming-guide.html#actions)_, which return values, and _[transformations](scala-programming-guide.html#transformations)_, which return pointers to new RDDs. Let's start with a few actions:
+
+{% highlight scala %}
+scala> textFile.count() // Number of items in this RDD
+res0: Long = 74
+
+scala> textFile.first() // First item in this RDD
+res1: String = # Spark
+{% endhighlight %}
+
+Now let's use a transformation. We will use the [`filter`](scala-programming-guide.html#transformations) transformation to return a new RDD with a subset of the items in the file.
+
+{% highlight scala %}
+scala> val linesWithSpark = textFile.filter(line => line.contains("Spark"))
+linesWithSpark: spark.RDD[String] = spark.FilteredRDD@7dd4af09
+{% endhighlight %}
+
+We can chain together transformations and actions:
+
+{% highlight scala %}
+scala> textFile.filter(line => line.contains("Spark")).count() // How many lines contain "Spark"?
+res3: Long = 15
+{% endhighlight %}
+
+## Transformations
+RDD transformations can be used for more complex computations. Let's say we want to find the line with the most words:
+
+{% highlight scala %}
+scala> textFile.map(line => line.split(" ").size).reduce((a, b) => if (a > b) a else b)
+res4: Long = 16
+{% endhighlight %}
+
+This first maps a line to an integer value, creating a new RDD. `reduce` is called on that RDD to find the largest line count. The arguments to `map` and `reduce` are Scala function literals (closures), and can use any language feature or Scala/Java library. For example, we can easily call functions declared elsewhere. We'll use `Math.max()` function to make this code easier to understand:
+
+{% highlight scala %}
+scala> import java.lang.Math
+import java.lang.Math
+
+scala> textFile.map(line => line.split(" ").size).reduce((a, b) => Math.max(a, b))
+res5: Int = 16
+{% endhighlight %}
+
+One common data flow pattern is MapReduce, as popularized by Hadoop. Spark can implement MapReduce flows easily:
+
+{% highlight scala %}
+scala> val wordCounts = textFile.flatMap(line => line.split(" ")).map(word => (word, 1)).reduceByKey((a, b) => a + b)
+wordCounts: spark.RDD[(java.lang.String, Int)] = spark.ShuffledAggregatedRDD@71f027b8
+{% endhighlight %}
+
+Here, we combined the [`flatMap`](scala-programming-guide.html#transformations), [`map`](scala-programming-guide.html#transformations) and [`reduceByKey`](scala-programming-guide.html#transformations) transformations to compute the per-word counts in the file as an RDD of (String, Int) pairs. To collect the word counts in our shell, we can use the [`collect`](scala-programming-guide.html#actions) action:
+
+{% highlight scala %}
+scala> wordCounts.collect()
+res6: Array[(java.lang.String, Int)] = Array((need,2), ("",43), (Extra,3), (using,1), (passed,1), (etc.,1), (its,1), (`/usr/local/lib/libmesos.so`,1), (`SCALA_HOME`,1), (option,1), (these,1), (#,1), (`PATH`,,2), (200,1), (To,3),...
+{% endhighlight %}
+
+## Caching
+Spark also supports pulling data sets into a cluster-wide in-memory cache. This is very useful when data is accessed repeatedly, such as when querying a small "hot" dataset or when running an iterative algorithm like PageRank. As a simple example, let's mark our `linesWithSpark` dataset to be cached:
+
+{% highlight scala %}
+scala> linesWithSpark.cache()
+res7: spark.RDD[String] = spark.FilteredRDD@17e51082
+
+scala> linesWithSpark.count()
+res8: Long = 15
+
+scala> linesWithSpark.count()
+res9: Long = 15
+{% endhighlight %}
+
+It may seem silly to use a Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark).
+
+# A Standalone Job in Scala
+Now say we wanted to write a standalone job using the Spark API. We will walk through a simple job in both Scala (with sbt) and Java (with maven). If you using other build systems, consider using the Spark assembly JAR described in the developer guide.
+
+We'll create a very simple Spark job in Scala. So simple, in fact, that it's named `SimpleJob.scala`:
+
+{% highlight scala %}
+/*** SimpleJob.scala ***/
+import spark.SparkContext
+import SparkContext._
+
+object SimpleJob extends Application {
+ val logFile = "/var/log/syslog" // Should be some file on your system
+ val sc = new SparkContext("local", "Simple Job", "$YOUR_SPARK_HOME",
+ "target/scala-{{site.SCALA_VERSION}}/simple-project_{{site.SCALA_VERSION}}-1.0.jar")
+ val logData = sc.textFile(logFile, 2).cache()
+ val numAs = logData.filter(line => line.contains("a")).count()
+ val numBs = logData.filter(line => line.contains("b")).count()
+ println("Lines with a: %s, Lines with b: %s".format(numAs, numBs))
+}
+{% endhighlight %}
+
+This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the job. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the job, the directory where Spark is installed, and a name for the jar file containing the job's sources. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+
+This 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 two repositories which host Spark dependencies:
+
+{% highlight scala %}
+name := "Simple Project"
+
+version := "1.0"
+
+scalaVersion := "{{site.SCALA_VERSION}}"
+
+libraryDependencies += "org.spark-project" %% "spark-core" % "{{site.SPARK_VERSION}}"
+
+resolvers ++= Seq(
+ "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
+ "Spray Repository" at "http://repo.spray.cc/")
+{% endhighlight %}
+
+Of course, for sbt to work correctly, we'll need to layout `SimpleJob.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the job's code, then use `sbt run` to execute our example job.
+
+{% highlight bash %}
+$ find .
+.
+./simple.sbt
+./src
+./src/main
+./src/main/scala
+./src/main/scala/SimpleJob.scala
+
+$ sbt package
+$ sbt run
+...
+Lines with a: 8422, Lines with b: 1836
+{% endhighlight %}
+
+This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS.
+
+# A Standalone Job In Java
+Now say we wanted to write a standalone job using the Java API. We will walk through doing this with Maven. If you using other build systems, consider using the Spark assembly JAR described in the developer guide.
+
+We'll create a very simple Spark job, `SimpleJob.java`:
+
+{% highlight java %}
+/*** SimpleJob.java ***/
+import spark.api.java.*;
+import spark.api.java.function.Function;
+
+public class SimpleJob {
+ public static void main(String[] args) {
+ String logFile = "/var/log/syslog"; // Should be some file on your system
+ JavaSparkContext sc = new JavaSparkContext("local", "Simple Job",
+ "$YOUR_SPARK_HOME", "target/simple-project-1.0.jar");
+ JavaRDD<String> logData = sc.textFile(logFile).cache();
+
+ long numAs = logData.filter(new Function<String, Boolean>() {
+ public Boolean call(String s) { return s.contains("a"); }
+ }).count();
+
+ long numBs = logData.filter(new Function<String, Boolean>() {
+ public Boolean call(String s) { return s.contains("b"); }
+ }).count();
+
+ System.out.println("Lines with a: " + numAs + ", lines with b: " + numBs);
+ }
+}
+{% endhighlight %}
+
+This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. Note that like in 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") describes these differences in more detail.
+
+To build the job, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version.
+
+{% highlight xml %}
+<project>
+ <groupId>edu.berkeley</groupId>
+ <artifactId>simple-project</artifactId>
+ <modelVersion>4.0.0</modelVersion>
+ <name>Simple Project</name>
+ <packaging>jar</packaging>
+ <version>1.0</version>
+ <dependencies>
+ <dependency> <!-- Spark dependency -->
+ <groupId>org.spark-project</groupId>
+ <artifactId>spark-core_{{site.SCALA_VERSION}}</artifactId>
+ <version>{{site.SPARK_VERSION}}</version>
+ </dependency>
+ </dependencies>
+</project>
+{% endhighlight %}
+
+We lay out these files according to the canonical Maven directory structure:
+{% highlight bash %}
+$ find .
+./pom.xml
+./src
+./src/main
+./src/main/java
+./src/main/java/SimpleJob.java
+{% endhighlight %}
+
+Now, we can execute the job using Maven:
+
+{% highlight bash %}
+$ mvn package
+$ mvn exec:java -Dexec.mainClass="SimpleJob"
+...
+Lines with a: 8422, Lines with b: 1836
+{% endhighlight %}
+
+This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS.
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
new file mode 100644
index 0000000000..97564d7426
--- /dev/null
+++ b/docs/running-on-mesos.md
@@ -0,0 +1,59 @@
+---
+layout: global
+title: Running Spark on Mesos
+---
+
+Spark can run on private clusters managed by the [Apache Mesos](http://incubator.apache.org/mesos/) resource manager. Follow the steps below to install Mesos and Spark:
+
+1. Download and build Spark using the instructions [here](index.html).
+2. Download Mesos {{site.MESOS_VERSION}} from a [mirror](http://www.apache.org/dyn/closer.cgi/incubator/mesos/mesos-{{site.MESOS_VERSION}}/).
+3. Configure Mesos using the `configure` script, passing the location of your `JAVA_HOME` using `--with-java-home`. Mesos comes with "template" configure scripts for different platforms, such as `configure.macosx`, that you can run. See the README file in Mesos for other options. **Note:** If you want to run Mesos without installing it into the default paths on your system (e.g. if you don't have administrative privileges to install it), you should also pass the `--prefix` option to `configure` to tell it where to install. For example, pass `--prefix=/home/user/mesos`. By default the prefix is `/usr/local`.
+4. Build Mesos using `make`, and then install it using `make install`.
+5. Create a file called `spark-env.sh` in Spark's `conf` directory, by copying `conf/spark-env.sh.template`, and add the following lines in it:
+ * `export MESOS_NATIVE_LIBRARY=<path to libmesos.so>`. This path is usually `<prefix>/lib/libmesos.so` (where the prefix is `/usr/local` by default). Also, on Mac OS X, the library is called `libmesos.dylib` instead of `.so`.
+ * `export SCALA_HOME=<path to Scala directory>`.
+6. Copy Spark and Mesos to the _same_ paths on all the nodes in the cluster (or, for Mesos, `make install` on every node).
+7. Configure Mesos for deployment:
+ * On your master node, edit `<prefix>/var/mesos/deploy/masters` to list your master and `<prefix>/var/mesos/deploy/slaves` to list the slaves, where `<prefix>` is the prefix where you installed Mesos (`/usr/local` by default).
+ * On all nodes, edit `<prefix>/var/mesos/deploy/mesos.conf` and add the line `master=HOST:5050`, where HOST is your master node.
+ * Run `<prefix>/sbin/mesos-start-cluster.sh` on your master to start Mesos. If all goes well, you should see Mesos's web UI on port 8080 of the master machine.
+ * See Mesos's README file for more information on deploying it.
+8. To run a Spark job against the cluster, when you create your `SparkContext`, pass the string `mesos://HOST:5050` as the first parameter, where `HOST` is the machine running your Mesos master. In addition, pass the location of Spark on your nodes as the third parameter, and a list of JAR files containing your JAR's code as the fourth (these will automatically get copied to the workers). For example:
+
+{% highlight scala %}
+new SparkContext("mesos://HOST:5050", "My Job Name", "/home/user/spark", List("my-job.jar"))
+{% 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,
+each Spark task runs as a separate Mesos task. This allows multiple instances of Spark (and other applications) to share
+machines at a very fine granularity, where each job gets more or fewer machines as it ramps up, but it comes with an
+additional overhead in launching each task, which may be inappropriate for low-latency applications that aim for
+sub-second Spark operations (e.g. interactive queries or serving web requests). The coarse-grained mode will instead
+launch only *one* long-running Spark task on each Mesos machine, and dynamically schedule its own "mini-tasks" within
+it. The benefit is much lower startup overhead, but at the cost of reserving the Mesos resources for the complete duration
+of the job.
+
+To run in coarse-grained mode, set the `spark.mesos.coarse` system property to true *before* creating your SparkContext:
+
+{% highlight scala %}
+System.setProperty("spark.mesos.coarse", "true")
+val sc = new SparkContext("mesos://HOST:5050", "Job Name", ...)
+{% endhighlight %}
+
+In addition, for coarse-grained mode, you can control the maximum number of resources Spark will acquire. By default,
+it will acquire *all* cores in the cluster (that get offered by Mesos), which only makes sense if you run just a single
+job at a time. You can cap the maximum number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
+Again, this must be done *before* initializing a SparkContext.
+
+
+# Running Alongside Hadoop
+
+You can run Spark and Mesos alongside your existing Hadoop cluster by just launching them as a separate service on the machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://<namenode>:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI).
+
+In addition, it is possible to also run Hadoop MapReduce on Mesos, to get better resource isolation and sharing between the two. In this case, Mesos will act as a unified scheduler that assigns cores to either Hadoop or Spark, as opposed to having them share resources via the Linux scheduler on each node. Please refer to the Mesos wiki page on [Running Hadoop on Mesos](https://github.com/mesos/mesos/wiki/Running-Hadoop-on-Mesos).
+
+In either case, HDFS runs separately from Hadoop MapReduce, without going through Mesos.
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
new file mode 100644
index 0000000000..6fb81b6004
--- /dev/null
+++ b/docs/running-on-yarn.md
@@ -0,0 +1,51 @@
+---
+layout: global
+title: Launching Spark on YARN
+---
+
+Experimental support for running over a [YARN (Hadoop
+NextGen)](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html)
+cluster was added to Spark in version 0.6.0. Because YARN depends on version
+2.0 of the Hadoop libraries, this currently requires checking out a separate
+branch of Spark, called `yarn`, which you can do as follows:
+
+ git clone git://github.com/mesos/spark
+ cd spark
+ git checkout -b yarn --track origin/yarn
+
+
+# Preparations
+
+- In order to distribute Spark within the cluster, it must be packaged into a single JAR file. This can be done by running `sbt/sbt assembly`
+- Your application code must be packaged into a separate JAR file.
+
+If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt package`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different.
+
+# Launching Spark on YARN
+
+The command to launch the YARN Client is as follows:
+
+ SPARK_JAR=<SPARK_YAR_FILE> ./run spark.deploy.yarn.Client \
+ --jar <YOUR_APP_JAR_FILE> \
+ --class <APP_MAIN_CLASS> \
+ --args <APP_MAIN_ARGUMENTS> \
+ --num-workers <NUMBER_OF_WORKER_MACHINES> \
+ --worker-memory <MEMORY_PER_WORKER> \
+ --worker-cores <CORES_PER_WORKER>
+
+For example:
+
+ SPARK_JAR=./core/target/spark-core-assembly-{{site.SPARK_VERSION}}.jar ./run spark.deploy.yarn.Client \
+ --jar examples/target/scala-{{site.SCALA_VERSION}}/spark-examples_{{site.SCALA_VERSION}}-{{site.SPARK_VERSION}}.jar \
+ --class spark.examples.SparkPi \
+ --args standalone \
+ --num-workers 3 \
+ --worker-memory 2g \
+ --worker-cores 2
+
+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.
+
+# Important Notes
+
+- When your application instantiates a Spark context it must use a special "standalone" master url. This starts the scheduler without forcing it to connect to a cluster. A good way to handle this is to pass "standalone" as an argument to your program, as shown in the example above.
+- YARN does not support requesting container resources based on the number of cores. Thus the numbers of cores given via command line arguments cannot be guaranteed.
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
new file mode 100644
index 0000000000..73f8b123be
--- /dev/null
+++ b/docs/scala-programming-guide.md
@@ -0,0 +1,354 @@
+---
+layout: global
+title: Spark Programming Guide
+---
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+
+# Overview
+
+At a high level, every Spark application consists of a *driver program* that runs the user's `main` function and executes various *parallel operations* on a cluster. The main abstraction Spark provides is a *resilient distributed dataset* (RDD), which is a collection of elements partitioned across the nodes of the cluster that can be operated on in parallel. RDDs are created by starting with a file in the Hadoop file system (or any other Hadoop-supported file system), or an existing Scala collection in the driver program, and transforming it. Users may also ask Spark to *persist* an RDD in memory, allowing it to be reused efficiently across parallel operations. Finally, RDDs automatically recover from node failures.
+
+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!
+
+# Linking with Spark
+
+To write a Spark application, you will need to add both Spark and its dependencies to your CLASSPATH. If you use sbt or Maven, Spark is available through Maven Central at:
+
+ groupId = org.spark_project
+ artifactId = spark-core_{{site.SCALA_VERSION}}
+ version = {{site.SPARK_VERSION}}
+
+For other build systems or environments, you can run `sbt/sbt assembly` to build both Spark and its dependencies into one JAR (`core/target/spark-core-assembly-0.6.0.jar`), then add this to your CLASSPATH.
+
+In addition, you'll need to import some Spark classes and implicit conversions. Add the following lines at the top of your program:
+
+{% highlight scala %}
+import spark.SparkContext
+import SparkContext._
+{% endhighlight %}
+
+# Initializing Spark
+
+The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark how to access a cluster.
+This is done through the following constructor:
+
+{% highlight scala %}
+new SparkContext(master, jobName, [sparkHome], [jars])
+{% endhighlight %}
+
+The `master` parameter is a string specifying a [Mesos](running-on-mesos.html) cluster to connect to, or a special "local" string to run in local mode, as described below. `jobName` is a name for your job, which will be shown in the Mesos web UI when running on a cluster. 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. For example, to run on four cores, use
+
+{% highlight bash %}
+$ MASTER=local[4] ./spark-shell
+{% endhighlight %}
+
+### Master URLs
+
+The master URL passed to Spark can be in one of the following formats:
+
+<table class="table">
+<tr><th>Master URL</th><th>Meaning</th></tr>
+<tr><td> local </td><td> Run Spark locally with one worker thread (i.e. no parallelism at all). </td></tr>
+<tr><td> local[K] </td><td> Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
+</td></tr>
+<tr><td> spark://HOST:PORT </td><td> Connect to the given <a href="spark-standalone.html">Spark standalone
+ cluster</a> master. The port must be whichever one your master is configured to use, which is 7077 by default.
+</td></tr>
+<tr><td> mesos://HOST:PORT </td><td> Connect to the given <a href="running-on-mesos.html">Mesos</a> cluster.
+ The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use,
+ which is 5050 by default.
+</td></tr>
+</table>
+
+For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details.
+
+### Deploying Code on a Cluster
+
+If you want to run your job on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code:
+
+* `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them).
+* `jars`: A list of JAR files on the local machine containing your job's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your job into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies.
+
+If you run `spark-shell` on a cluster, any classes you define in the shell will automatically be distributed.
+
+
+# Resilient Distributed Datasets (RDDs)
+
+Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods.
+
+## Parallelized Collections
+
+Parallelized collections are created by calling `SparkContext`'s `parallelize` method on an existing Scala collection (a `Seq` object). The elements of the collection are copied to form a distributed dataset that can be operated on in parallel. For example, here is some interpreter output showing how to create a parallel collection from an array:
+
+{% highlight scala %}
+scala> val data = Array(1, 2, 3, 4, 5)
+data: Array[Int] = Array(1, 2, 3, 4, 5)
+
+scala> val distData = sc.parallelize(data)
+distData: spark.RDD[Int] = spark.ParallelCollection@10d13e3e
+{% endhighlight %}
+
+Once created, the distributed dataset (`distData` here) can be operated on in parallel. For example, we might call `distData.reduce(_ + _)` to add up the elements of the array. We describe operations on distributed datasets later on.
+
+One important parameter for parallel collections is the number of *slices* to cut the dataset into. Spark will run one task for each slice of the cluster. Typically you want 2-4 slices for each CPU in your cluster. Normally, Spark tries to set the number of slices automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`).
+
+## Hadoop Datasets
+
+Spark can create distributed datasets from any file stored in the Hadoop distributed file system (HDFS) or other storage systems supported by Hadoop (including your local file system, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), Hypertable, HBase, etc). Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop InputFormat.
+
+Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3n://`, `kfs://`, etc URI). Here is an example invocation:
+
+{% highlight scala %}
+scala> val distFile = sc.textFile("data.txt")
+distFile: spark.RDD[String] = spark.HadoopRDD@1d4cee08
+{% endhighlight %}
+
+Once created, `distFile` can be acted on by dataset operations. For example, we can add up the sizes of all the lines using the `map` and `reduce` operations as follows: `distFile.map(_.size).reduce(_ + _)`.
+
+The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks.
+
+For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). In addition, Spark allows you to specify native types for a few common Writables; for example, `sequenceFile[Int, String]` will automatically read IntWritables and Texts.
+
+Finally, for other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source.
+
+## RDD Operations
+
+RDDs support two types of operations: *transformations*, which create a new dataset from an existing one, and *actions*, which return a value to the driver program after running a computation on the dataset. For example, `map` is a transformation that passes each dataset element through a function and returns a new distributed dataset representing the results. On the other hand, `reduce` is an action that aggregates all the elements of the dataset using some function and returns the final result to the driver program (although there is also a parallel `reduceByKey` that returns a distributed dataset).
+
+All transformations in Spark are <i>lazy</i>, in that they do not compute their results right away. Instead, they just remember the transformations applied to some base dataset (e.g. a file). The transformations are only computed when an action requires a result to be returned to the driver program. This design enables Spark to run more efficiently -- for example, we can realize that a dataset created through `map` will be used in a `reduce` and return only the result of the `reduce` to the driver, rather than the larger mapped dataset.
+
+By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options.
+
+The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#spark.RDD) for details):
+
+### Transformations
+
+<table class="table">
+<tr><th style="width:25%">Transformation</th><th>Meaning</th></tr>
+<tr>
+ <td> <b>map</b>(<i>func</i>) </td>
+ <td> Return a new distributed dataset formed by passing each element of the source through a function <i>func</i>. </td>
+</tr>
+<tr>
+ <td> <b>filter</b>(<i>func</i>) </td>
+ <td> Return a new dataset formed by selecting those elements of the source on which <i>func</i> returns true. </td>
+</tr>
+<tr>
+ <td> <b>flatMap</b>(<i>func</i>) </td>
+ <td> Similar to map, but each input item can be mapped to 0 or more output items (so <i>func</i> should return a Seq rather than a single item). </td>
+</tr>
+<tr>
+ <td> <b>mapPartitions</b>(<i>func</i>) </td>
+ <td> Similar to map, but runs separately on each partition (block) of the RDD, so <i>func</i> must be of type
+ Iterator[T] => Iterator[U] when running on an RDD of type T. </td>
+</tr>
+<tr>
+ <td> <b>mapPartitionsWithSplit</b>(<i>func</i>) </td>
+ <td> Similar to mapPartitions, but also provides <i>func</i> with an integer value representing the index of
+ the split, so <i>func</i> must be of type (Int, Iterator[T]) => Iterator[U] when running on an RDD of type T.
+ </td>
+</tr>
+<tr>
+ <td> <b>sample</b>(<i>withReplacement</i>, <i>fraction</i>, <i>seed</i>) </td>
+ <td> Sample a fraction <i>fraction</i> of the data, with or without replacement, using a given random number generator seed. </td>
+</tr>
+<tr>
+ <td> <b>union</b>(<i>otherDataset</i>) </td>
+ <td> Return a new dataset that contains the union of the elements in the source dataset and the argument. </td>
+</tr>
+<tr>
+ <td> <b>distinct</b>([<i>numTasks</i>])) </td>
+ <td> Return a new dataset that contains the distinct elements of the source dataset.</td>
+</tr>
+<tr>
+ <td> <b>groupByKey</b>([<i>numTasks</i>]) </td>
+ <td> When called on a dataset of (K, V) pairs, returns a dataset of (K, Seq[V]) pairs. <br />
+<b>Note:</b> By default, this uses only 8 parallel tasks to do the grouping. You can pass an optional <code>numTasks</code> argument to set a different number of tasks.
+</td>
+</tr>
+<tr>
+ <td> <b>reduceByKey</b>(<i>func</i>, [<i>numTasks</i>]) </td>
+ <td> When called on a dataset of (K, V) pairs, returns a dataset of (K, V) pairs where the values for each key are aggregated using the given reduce function. Like in <code>groupByKey</code>, the number of reduce tasks is configurable through an optional second argument. </td>
+</tr>
+<tr>
+ <td> <b>sortByKey</b>([<i>ascending</i>], [<i>numTasks</i>]) </td>
+ <td> When called on a dataset of (K, V) pairs where K implements Ordered, returns a dataset of (K, V) pairs sorted by keys in ascending or descending order, as specified in the boolean <code>ascending</code> argument.</td>
+</tr>
+<tr>
+ <td> <b>join</b>(<i>otherDataset</i>, [<i>numTasks</i>]) </td>
+ <td> When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (V, W)) pairs with all pairs of elements for each key. </td>
+</tr>
+<tr>
+ <td> <b>cogroup</b>(<i>otherDataset</i>, [<i>numTasks</i>]) </td>
+ <td> When called on datasets of type (K, V) and (K, W), returns a dataset of (K, Seq[V], Seq[W]) tuples. This operation is also called <code>groupWith</code>. </td>
+</tr>
+<tr>
+ <td> <b>cartesian</b>(<i>otherDataset</i>) </td>
+ <td> When called on datasets of types T and U, returns a dataset of (T, U) pairs (all pairs of elements). </td>
+</tr>
+</table>
+
+A complete list of transformations is available in the [RDD API doc](api/core/index.html#spark.RDD).
+
+### Actions
+
+<table class="table">
+<tr><th>Action</th><th>Meaning</th></tr>
+<tr>
+ <td> <b>reduce</b>(<i>func</i>) </td>
+ <td> Aggregate the elements of the dataset using a function <i>func</i> (which takes two arguments and returns one). The function should be associative so that it can be computed correctly in parallel. </td>
+</tr>
+<tr>
+ <td> <b>collect</b>() </td>
+ <td> Return all the elements of the dataset as an array at the driver program. This is usually useful after a filter or other operation that returns a sufficiently small subset of the data. </td>
+</tr>
+<tr>
+ <td> <b>count</b>() </td>
+ <td> Return the number of elements in the dataset. </td>
+</tr>
+<tr>
+ <td> <b>first</b>() </td>
+ <td> Return the first element of the dataset (similar to take(1)). </td>
+</tr>
+<tr>
+ <td> <b>take</b>(<i>n</i>) </td>
+ <td> Return an array with the first <i>n</i> elements of the dataset. Note that this is currently not executed in parallel. Instead, the driver program computes all the elements. </td>
+</tr>
+<tr>
+ <td> <b>takeSample</b>(<i>withReplacement</i>, <i>num</i>, <i>seed</i>) </td>
+ <td> Return an array with a random sample of <i>num</i> elements of the dataset, with or without replacement, using the given random number generator seed. </td>
+</tr>
+<tr>
+ <td> <b>saveAsTextFile</b>(<i>path</i>) </td>
+ <td> Write the elements of the dataset as a text file (or set of text files) in a given directory in the local filesystem, HDFS or any other Hadoop-supported file system. Spark will call toString on each element to convert it to a line of text in the file. </td>
+</tr>
+<tr>
+ <td> <b>saveAsSequenceFile</b>(<i>path</i>) </td>
+ <td> Write the elements of the dataset as a Hadoop SequenceFile in a given path in the local filesystem, HDFS or any other Hadoop-supported file system. This is only available on RDDs of key-value pairs that either implement Hadoop's Writable interface or are implicitly convertible to Writable (Spark includes conversions for basic types like Int, Double, String, etc). </td>
+</tr>
+<tr>
+ <td> <b>countByKey</b>() </td>
+ <td> Only available on RDDs of type (K, V). Returns a `Map` of (K, Int) pairs with the count of each key. </td>
+</tr>
+<tr>
+ <td> <b>foreach</b>(<i>func</i>) </td>
+ <td> Run a function <i>func</i> on each element of the dataset. This is usually done for side effects such as updating an accumulator variable (see below) or interacting with external storage systems. </td>
+</tr>
+</table>
+
+A complete list of actions is available in the [RDD API doc](api/core/index.html#spark.RDD).
+
+## RDD Persistence
+
+One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory across operations. When you persist an RDD, each node stores any slices of it that it computes in memory and reuses them in other actions on that dataset (or datasets derived from it). This allows future actions to be much faster (often by more than 10x). Caching is a key tool for building iterative algorithms with Spark and for interactive use from the interpreter.
+
+You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it.
+
+In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`spark.storage.StorageLevel`](api/core/index.html#spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is:
+
+<table class="table">
+<tr><th style="width:23%">Storage Level</th><th>Meaning</th></tr>
+<tr>
+ <td> MEMORY_ONLY </td>
+ <td> Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, some partitions will
+ not be cached and will be recomputed on the fly each time they're needed. This is the default level. </td>
+</tr>
+<tr>
+ <td> MEMORY_AND_DISK </td>
+ <td> Store RDD as deserialized Java objects in the JVM. If the RDD does not fit in memory, store the
+ partitions that don't fit on disk, and read them from there when they're needed. </td>
+</tr>
+<tr>
+ <td> MEMORY_ONLY_SER </td>
+ <td> Store RDD as <i>serialized</i> Java objects (one byte array per partition).
+ This is generally more space-efficient than deserialized objects, especially when using a
+ <a href="tuning.html">fast serializer</a>, but more CPU-intensive to read.
+ </td>
+</tr>
+<tr>
+ <td> MEMORY_AND_DISK_SER </td>
+ <td> Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of recomputing them
+ on the fly each time they're needed. </td>
+</tr>
+<tr>
+ <td> DISK_ONLY </td>
+ <td> Store the RDD partitions only on disk. </td>
+</tr>
+<tr>
+ <td> MEMORY_ONLY_2, MEMORY_AND_DISK_2, etc. </td>
+ <td> Same as the levels above, but replicate each partition on two cluster nodes. </td>
+</tr>
+</table>
+
+### Which Storage Level to Choose?
+
+Spark's storage levels are meant to provide different tradeoffs between memory usage and CPU efficiency.
+We recommend going through the following process to select one:
+
+* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most
+ CPU-efficient option, allowing operations on the RDDs to run as fast as possible.
+* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to make the objects
+ much more space-efficient, but still reasonably fast to access.
+* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter a large
+ amount of the data. Otherwise, recomputing a partition is about as fast as reading it from disk.
+* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web
+ application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones
+ let you continue running tasks on the RDD without waiting to recompute a lost partition.
+
+
+# Shared Variables
+
+Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a remote cluster node, it works on separate copies of all the variables used in the function. These variables are copied to each machine, and no updates to the variables on the remote machine are propagated back to the driver program. Supporting general, read-write shared variables across tasks would be inefficient. However, Spark does provide two limited types of *shared variables* for two common usage patterns: broadcast variables and accumulators.
+
+## Broadcast Variables
+
+Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables using efficient broadcast algorithms to reduce communication cost.
+
+Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` method. The interpreter session below shows this:
+
+{% highlight scala %}
+scala> val broadcastVar = sc.broadcast(Array(1, 2, 3))
+broadcastVar: spark.Broadcast[Array[Int]] = spark.Broadcast(b5c40191-a864-4c7d-b9bf-d87e1a4e787c)
+
+scala> broadcastVar.value
+res0: Array[Int] = Array(1, 2, 3)
+{% endhighlight %}
+
+After the broadcast variable is created, it should be used instead of the value `v` in any functions run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object `v` should not be modified after it is broadcast in order to ensure that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped to a new node later).
+
+## Accumulators
+
+Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of type Int and Double, and programmers can add support for new types.
+
+An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks running on the cluster can then add to it using the `+=` operator. However, they cannot read its value. Only the driver program can read the accumulator's value, using its `value` method.
+
+The interpreter session below shows an accumulator being used to add up the elements of an array:
+
+{% highlight scala %}
+scala> val accum = sc.accumulator(0)
+accum: spark.Accumulator[Int] = 0
+
+scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x)
+...
+10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s
+
+scala> accum.value
+res2: Int = 10
+{% endhighlight %}
+
+
+# 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 sample programs 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` script included in Spark -- for example, `./run spark.examples.SparkPi`. Each example program prints usage help when run without any arguments.
+
+For help on optimizing your program, the [configuration](configuration.html) and
+[tuning](tuning.html) guides provide information on best practices. They are especially important for
+making sure that your data is stored in memory in an efficient format.
diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md
new file mode 100644
index 0000000000..f6f0988858
--- /dev/null
+++ b/docs/spark-debugger.md
@@ -0,0 +1,121 @@
+---
+layout: global
+title: The Spark Debugger
+---
+**Summary:** The Spark debugger provides replay debugging for deterministic (logic) errors in Spark programs. It's currently in development, but you can try it out in the [arthur branch](https://github.com/mesos/spark/tree/arthur).
+
+## Introduction
+
+From a user's point of view, debugging a general distributed program can be tedious and confusing. Many distributed programs are nondeterministic; their outcome depends on the interleaving between computation and message passing across multiple machines. Also, the fact that a program is running on a cluster of hundreds or thousands of machines means that it's hard to understand the program state and pinpoint the location of problems.
+
+In order to tame nondeterminism, a distributed debugger has to log a lot of information, imposing a serious performance penalty on the application being debugged.
+
+But the Spark programming model lets us provide replay debugging for almost zero overhead. Spark programs are a series of RDDs and deterministic transformations, so when debugging a Spark program, we don't have to debug it all at once -- instead, we can debug each transformation individually. Broadly, the debugger lets us do the following two things:
+
+* Recompute and inspect intermediate RDDs after the program has finished.
+* Re-run a particular task in a single-threaded debugger to find exactly what went wrong.
+
+For deterministic errors, debugging a Spark program is now as easy as debugging a single-threaded one.
+
+## Approach
+
+As your Spark program runs, the slaves report key events back to the master -- for example, RDD creations, RDD contents, and uncaught exceptions. (A full list of event types is in [EventLogging.scala](https://github.com/mesos/spark/blob/arthur/core/src/main/scala/spark/EventLogging.scala).) The master logs those events, and you can load the event log into the debugger after your program is done running.
+
+_A note on nondeterminism:_ For fault recovery, Spark requires RDD transformations (for example, the function passed to `RDD.map`) to be deterministic. The Spark debugger also relies on this property, and it can also warn you if your transformation is nondeterministic. This works by checksumming the contents of each RDD and comparing the checksums from the original execution to the checksums after recomputing the RDD in the debugger.
+
+## Usage
+
+### Enabling the event log
+
+To turn on event logging for your program, set `$SPARK_JAVA_OPTS` in `conf/spark-env.sh` as follows:
+
+{% highlight bash %}
+export SPARK_JAVA_OPTS='-Dspark.arthur.logPath=path/to/event-log'
+{% endhighlight %}
+
+where `path/to/event-log` is where you want the event log to go relative to `$SPARK_HOME`.
+
+**Warning:** If `path/to/event-log` already exists, event logging will be automatically disabled.
+
+### Loading the event log into the debugger
+
+1. Run a Spark shell with `MASTER=<i>host</i> ./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"))
+r: spark.EventLogReader = spark.EventLogReader@726b37ad
+{% endhighlight %}
+
+ **Warning:** If the event log doesn't exist or is unreadable, this will silently fail and `r.events` will be empty.
+
+### Exploring intermediate RDDs
+
+Use `r.rdds` to get a list of intermediate RDDs generated during your program's execution. An RDD with id _x_ is located at <code>r.rdds(<i>x</i>)</code>. For example:
+
+{% highlight scala %}
+scala> r.rdds
+res8: scala.collection.mutable.ArrayBuffer[spark.RDD[_]] = ArrayBuffer(spark.HadoopRDD@fe85adf, spark.MappedRDD@5fa5eea1, spark.MappedRDD@6d5bd16, spark.ShuffledRDD@3a70f2db, spark.FlatMappedValuesRDD@4d5825d6, spark.MappedValuesRDD@561c2c45, spark.CoGroupedRDD@539e922d, spark.MappedValuesRDD@4f8ef33e, spark.FlatMappedRDD@32039440, spark.ShuffledRDD@8fa0f67, spark.MappedValuesRDD@590937cb, spark.CoGroupedRDD@6c2e1e17, spark.MappedValuesRDD@47b9af7d, spark.FlatMappedRDD@6fb05c54, spark.ShuffledRDD@237dc815, spark.MappedValuesRDD@16daece7, spark.CoGroupedRDD@7ef73d69, spark.MappedValuesRDD@19e0f99e, spark.FlatMappedRDD@1240158, spark.ShuffledRDD@62d438fd, spark.MappedValuesRDD@5ae99cbb, spark.FilteredRDD@1f30e79e, spark.MappedRDD@43b64611)
+{% endhighlight %}
+
+Use `r.printRDDs()` to get a formatted list of intermediate RDDs, along with the source location where they were created. For example:
+
+{% highlight scala %}
+scala> r.printRDDs
+#00: HadoopRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31)
+#01: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31)
+#02: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
+#03: ShuffledRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
+#04: FlatMappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
+#05: MappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:91)
+#06: CoGroupedRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:92)
+[...]
+{% endhighlight %}
+
+Use `r.visualizeRDDs()` to visualize the RDDs as a dependency graph. For example:
+
+{% highlight scala %}
+scala> r.visualizeRDDs
+/tmp/spark-rdds-3758182885839775712.pdf
+{% endhighlight %}
+
+![Example RDD dependency graph](http://www.ankurdave.com/images/rdd-dep-graph.png)
+
+Iterate over the `RDDCreation` entries in `r.events` (e.g. `for (RDDCreation(rdd, location) <- events)`) to access the RDD creation locations as well as the RDDs themselves.
+
+### Debugging a particular task
+
+1. Find the task you want to debug. If the task threw an exception, the `ExceptionEvent` that was created will have a reference to the task. For example:
+ {% highlight scala %}
+spark> val task = r.events.collect { case e: ExceptionEvent => e }.head.task
+{% endhighlight %}
+ Otherwise, look through the list of all tasks in `r.tasks`, or browse tasks by RDD using <code>r.tasksForRDD(<i>rdd</i>)</code>, which returns a list of tasks whose input is the given RDD.
+
+2. Run the task by calling <code>r.debugTask(<i>taskStageId</i>, <i>taskPartition</i>)</code>. The task should contain these two values; you can extract them as follows:
+ {% highlight scala %}
+val (taskStageId, taskPartition) = task match {
+ case rt: ResultTask[_, _] => (rt.stageId, rt.partition)
+ case smt: ShuffleMapTask => (smt.stageId, smt.partition)
+ case _ => throw new UnsupportedOperationException
+})
+{% endhighlight %}
+ The Spark debugger will launch the task in a separate JVM, but you will see the task's stdout and stderr inline with the Spark shell. If you want to pass custom debugging arguments to the task's JVM (for example, to change the debugging port), set the optional `debugOpts` argument to `r.debugTask`. When `debugOpts` is left unset, it defaults to:
+ {% highlight scala %}
+-Xdebug -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=8000
+{% endhighlight %}
+
+3. In another terminal, attach your favorite conventional debugger to the Spark shell. For example, if you want to use jdb, run `jdb -attach 8000`.
+
+4. Debug the task as you would debug a normal program. For example, to break when an exception is thrown:
+ {% highlight scala %}
+> catch org.xml.sax.SAXParseException
+{% endhighlight %}
+
+5. When the task ends, its JVM will quit and control will return to the main Spark shell. To stop it prematurely, you can kill it from the debugger, or interrupt it from the terminal with Ctrl-C.
+
+### Detecting nondeterminism in your transformations
+
+When a task gets run more than once, Arthur is able to compare the checksums of the task's output. If they are different, Arthur will insert a `ChecksumEvent` into `r.checksumMismatches` and print a warning like the following:
+ {% highlight scala %}
+12/04/07 11:42:44 WARN spark.EventLogWriter: Nondeterminism detected in shuffle output on RDD 2, partition 3, output split 0
+{% endhighlight %}
+
diff --git a/docs/spark-simple-tutorial.md b/docs/spark-simple-tutorial.md
new file mode 100644
index 0000000000..9875de62bd
--- /dev/null
+++ b/docs/spark-simple-tutorial.md
@@ -0,0 +1,41 @@
+---
+layout: global
+title: Tutorial - Running a Simple Spark Application
+---
+
+1. Create directory for spark demo:
+
+ ~$ mkdir SparkTest
+
+2. Copy the sbt files in ~/spark/sbt directory:
+
+ ~/SparkTest$ cp -r ../spark/sbt .
+
+3. Edit the ~/SparkTest/sbt/sbt file to look like this:
+
+ #!/bin/bash
+ java -Xmx800M -XX:MaxPermSize=150m -jar $(dirname $0)/sbt-launch-*.jar "$@"
+
+4. To build a Spark application, you need Spark and its dependencies in a single Java archive (JAR) file. Create this JAR in Spark's main directory with sbt as:
+
+ ~/spark$ sbt/sbt assembly
+
+5. create a source file in ~/SparkTest/src/main/scala directory:
+
+ ~/SparkTest/src/main/scala$ vi Test1.scala
+
+6. Make the contain of the Test1.scala file like this:
+
+ import spark.SparkContext
+ import spark.SparkContext._
+ object Test1 {
+ def main(args: Array[String]) {
+ val sc = new SparkContext("local", "SparkTest")
+ println(sc.parallelize(1 to 10).reduce(_ + _))
+ System.exit(0)
+ }
+ }
+
+7. Run the Test1.scala file:
+
+ ~/SparkTest$ sbt/sbt run
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
new file mode 100644
index 0000000000..ae630a0371
--- /dev/null
+++ b/docs/spark-standalone.md
@@ -0,0 +1,152 @@
+---
+layout: global
+title: Spark Standalone Mode
+---
+
+{% comment %}
+TODO(andyk):
+ - Add a table of contents
+ - Move configuration towards the end so that it doesn't come first
+ - Say the scripts will guess the resource amounts (i.e. # cores) automatically
+{% endcomment %}
+
+In addition to running on top of [Mesos](https://github.com/mesos/mesos), Spark also supports a standalone mode, consisting of one Spark master and several Spark worker processes. You can run the Spark standalone mode either locally (for testing) or on a cluster. If you wish to run on a cluster, we have provided [a set of deploy scripts](#cluster-launch-scripts) to launch a whole cluster.
+
+# Getting Started
+
+Compile Spark with `sbt package` as described in the [Getting Started Guide](index.html). You do not need to install Mesos on your machine if you are using the standalone mode.
+
+# Starting a Cluster Manually
+
+You can start a standalone master server by executing:
+
+ ./run spark.deploy.master.Master
+
+Once started, the master will print out a `spark://IP:PORT` URL for itself, which you can use to connect workers to it,
+or pass as the "master" argument to `SparkContext` to connect a job to the cluster. You can also find this URL on
+the master's web UI, which is [http://localhost:8080](http://localhost:8080) by default.
+
+Similarly, you can start one or more workers and connect them to the master via:
+
+ ./run 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).
+
+Finally, the following configuration options can be passed to the master and worker:
+
+<table class="table">
+ <tr><th style="width:21%">Argument</th><th>Meaning</th></tr>
+ <tr>
+ <td><code>-i IP</code>, <code>--ip IP</code></td>
+ <td>IP address or DNS name to listen on</td>
+ </tr>
+ <tr>
+ <td><code>-p PORT</code>, <code>--port PORT</code></td>
+ <td>IP address or DNS name to listen on (default: 7077 for master, random for worker)</td>
+ </tr>
+ <tr>
+ <td><code>--webui-port PORT</code></td>
+ <td>Port for web UI (default: 8080 for master, 8081 for worker)</td>
+ </tr>
+ <tr>
+ <td><code>-c CORES</code>, <code>--cores CORES</code></td>
+ <td>Number of CPU cores to use (default: all available); only on worker</td>
+ </tr>
+ <tr>
+ <td><code>-m MEM</code>, <code>--memory MEM</code></td>
+ <td>Amount of memory to use, 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>
+ <td>Directory to use for scratch space and job output logs (default: SPARK_HOME/work); only on worker</td>
+ </tr>
+</table>
+
+
+# Cluster Launch Scripts
+
+To launch a Spark standalone cluster with the deploy scripts, you need to set up two files, `conf/spark-env.sh` and `conf/slaves`. The `conf/spark-env.sh` file lets you specify global settings for the master and slave instances, such as memory, or port numbers to bind to, while `conf/slaves` is a list of slave nodes. The system requires that all the slave machines have the same configuration files, so *copy these files to each machine*.
+
+In `conf/spark-env.sh`, you can set the following parameters, in addition to the [standard Spark configuration settongs](configuration.html):
+
+<table class="table">
+ <tr><th style="width:21%">Environment Variable</th><th>Meaning</th></tr>
+ <tr>
+ <td><code>SPARK_MASTER_IP</code></td>
+ <td>Bind the master to a specific IP address, for example a public one</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_MASTER_PORT</code></td>
+ <td>Start the master on a different port (default: 7077)</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_MASTER_WEBUI_PORT</code></td>
+ <td>Port for the master web UI (default: 8080)</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_WORKER_PORT</code></td>
+ <td>Start the Spark worker on a specific port (default: random)</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_WORKER_CORES</code></td>
+ <td>Number of cores to use (default: all available cores)</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_WORKER_MEMORY</code></td>
+ <td>How much memory to use, e.g. 1000M, 2G (default: total memory minus 1 GB)</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_WORKER_WEBUI_PORT</code></td>
+ <td>Port for the worker web UI (default: 8081)</td>
+ </tr>
+ <tr>
+ <td><code>SPARK_WORKER_DIR</code></td>
+ <td>Directory to run jobs in, which will include both logs and scratch space (default: SPARK_HOME/work)</td>
+ </tr>
+</table>
+
+In `conf/slaves`, include a list of all machines where you would like to start a Spark worker, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing purposes, you can have a single `localhost` entry in the slaves file.
+
+Once you've set up these configuration files, 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.
+
+Note that the scripts must be executed on the machine you want to run the Spark master on, not your local machine.
+
+
+# Connecting a Job to the Cluster
+
+To run a job on the Spark cluster, simply pass the `spark://IP:PORT` URL of the master as to the [`SparkContext`
+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
+
+
+# Job Scheduling
+
+The standalone cluster mode currently only supports a simple FIFO scheduler across jobs.
+However, to allow multiple concurrent jobs, you can control the maximum number of resources each Spark job will acquire.
+By default, it will acquire *all* the cores in the cluster, which only makes sense if you run just a single
+job at a time. You can cap the number of cores using `System.setProperty("spark.cores.max", "10")` (for example).
+This value must be set *before* initializing your SparkContext.
+
+
+# Monitoring and Logging
+
+Spark's standalone mode offers a web-based user interface to monitor the cluster. The master and each worker has its own web UI that shows cluster and job statistics. By default you can access the web UI for the master at port 8080. The port can be changed either in the configuration file or via command-line options.
+
+In addition, detailed log output for each job is also written to the work directory of each slave node (`SPARK_HOME/work` by default). You will see two files for each job, `stdout` and `stderr`, with all output it wrote to its console.
+
+
+# Running Alongside Hadoop
+
+You can run Spark alongside your existing Hadoop cluster by just launching it as a separate service on the machines. To access Hadoop data from Spark, just use a hdfs:// URL (typically `hdfs://<namenode>:9000/path`, but you can find the right URL on your Hadoop Namenode's web UI). Alternatively, you can set up a separate cluster for Spark, and still have it access HDFS over the network; this will be slower than disk-local access, but may not be a concern if you are still running in the same local area network (e.g. you place a few Spark machines on each rack that you have Hadoop on).
+
diff --git a/docs/tuning.md b/docs/tuning.md
new file mode 100644
index 0000000000..f18de8ff3a
--- /dev/null
+++ b/docs/tuning.md
@@ -0,0 +1,248 @@
+---
+layout: global
+title: Tuning Spark
+---
+
+* This will become a table of contents (this text will be scraped).
+{:toc}
+
+Because of the in-memory nature of most Spark computations, Spark programs can be bottlenecked
+by any resource in the cluster: CPU, network bandwidth, or memory.
+Most often, if the data fits in memory, the bottleneck is network bandwidth, but sometimes, you
+also need to do some tuning, such as
+[storing RDDs in serialized form](scala-programming-guide.html#rdd-persistence), to
+decrease memory usage.
+This guide will cover two main topics: data serialization, which is crucial for good network
+performance and can also reduce memory use, and memory tuning. We also sketch several smaller topics.
+
+# Data Serialization
+
+Serialization plays an important role in the performance of any distributed application.
+Formats that are slow to serialize objects into, or consume a large number of
+bytes, will greatly slow down the computation.
+Often, this will be the first thing you should tune to optimize a Spark application.
+Spark aims to strike a balance between convenience (allowing you to work with any Java type
+in your operations) and performance. It provides two serialization libraries:
+
+* [Java serialization](http://docs.oracle.com/javase/6/docs/api/java/io/Serializable.html):
+ By default, Spark serializes objects using Java's `ObjectOutputStream` framework, and can work
+ with any class you create that implements
+ [`java.io.Serializable`](http://docs.oracle.com/javase/6/docs/api/java/io/Serializable.html).
+ You can also control the performance of your serialization more closely by extending
+ [`java.io.Externalizable`](http://docs.oracle.com/javase/6/docs/api/java/io/Externalizable.html).
+ Java serialization is flexible but often quite slow, and leads to large
+ serialized formats for many classes.
+* [Kryo serialization](http://code.google.com/p/kryo/wiki/V1Documentation): Spark can also use
+ the Kryo library (currently just version 1) to serialize objects more quickly. Kryo is significantly
+ faster and more compact than Java serialization (often as much as 10x), but does not support all
+ `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", "spark.KryoSerializer")`
+*before* creating your SparkContext. The only reason it 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
+[`spark.KryoRegistrator`](api/core/index.html#spark.KryoRegistrator) and set the
+`spark.kryo.registrator` system property to point to it, as follows:
+
+{% highlight scala %}
+class MyRegistrator extends KryoRegistrator {
+ override def registerClasses(kryo: Kryo) {
+ kryo.register(classOf[MyClass1])
+ kryo.register(classOf[MyClass2])
+ }
+}
+
+// Make sure to set these properties *before* creating a SparkContext!
+System.setProperty("spark.serializer", "spark.KryoSerializer")
+System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator")
+val sc = new SparkContext(...)
+{% endhighlight %}
+
+The [Kryo documentation](http://code.google.com/p/kryo/wiki/V1Documentation) 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 32, 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
+full class name with each object, which is wasteful.
+
+# Memory Tuning
+
+There are three considerations in tuning memory usage: the *amount* of memory used by your objects
+(you may want your entire dataset to fit in memory), the *cost* of accessing those objects, and the
+overhead of *garbage collection* (if you have high turnover in terms of objects).
+
+By default, Java objects are fast to access, but can easily consume a factor of 2-5x more space
+than the "raw" data inside their fields. This is due to several reasons:
+
+* Each distinct Java object has an "object header", which is about 16 bytes and contains information
+ such as a pointer to its class. For an object with very little data in it (say one `Int` field), this
+ can be bigger than the data.
+* Java Strings have about 40 bytes of overhead over the raw string data (since they store it in an
+ array of `Char`s and keep extra data such as the length), and store each character
+ as *two* bytes due to Unicode. Thus a 10-character string can easily consume 60 bytes.
+* Common collection classes, such as `HashMap` and `LinkedList`, use linked data structures, where
+ there is a "wrapper" object for each entry (e.g. `Map.Entry`). This object not only has a header,
+ but also pointers (typically 8 bytes each) to the next object in the list.
+* Collections of primitive types often store them as "boxed" objects such as `java.lang.Integer`.
+
+This section will discuss how to determine the memory usage of your objects, and how to improve
+it -- either by changing your data structures, or by storing data in a serialized format.
+We will then cover tuning Spark's cache size and the Java garbage collector.
+
+## Determining Memory Consumption
+
+The best way to size the amount of memory consumption your dataset will require is to create an RDD, put it into cache, and look at the SparkContext logs on your driver program. The logs will tell you how much memory each partition is consuming, which you can aggregate to get the total size of the RDD. You will see messages like this:
+
+ INFO BlockManagerMasterActor: Added rdd_0_1 in memory on mbk.local:50311 (size: 717.5 KB, free: 332.3 MB)
+
+This means that partition 1 of RDD 0 consumed 717.5 KB.
+
+## Tuning Data Structures
+
+The first way to reduce memory consumption is to avoid the Java features that add overhead, such as
+pointer-based data structures and wrapper objects. There are several ways to do this:
+
+1. Design your data structures to prefer arrays of objects, and primitive types, instead of the
+ standard Java or Scala collection classes (e.g. `HashMap`). The [fastutil](http://fastutil.di.unimi.it)
+ library provides convenient collection classes for primitive types that are compatible with the
+ Java standard library.
+2. Avoid nested structures with a lot of small objects and pointers when possible.
+3. Consider using numeric IDs or enumeration objects instead of strings for keys.
+4. If you have less than 32 GB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be
+ four bytes instead of eight. Also, on Java 7 or later, try `-XX:+UseCompressedStrings` to store
+ ASCII strings as just 8 bits per character. You can add these options in
+ [`spark-env.sh`](configuration.html#environment-variables-in-spark-envsh).
+
+## Serialized RDD Storage
+
+When your objects are still too large to efficiently store despite this tuning, a much simpler way
+to reduce memory usage is to store them in *serialized* form, using the serialized StorageLevels in
+the [RDD persistence API](scala-programming-guide.html#rdd-persistence), such as `MEMORY_ONLY_SER`.
+Spark will then store each RDD partition as one large byte array.
+The only downside of storing data in serialized form is slower access times, due to having to
+deserialize each object on the fly.
+We highly recommend [using Kryo](#data-serialization) if you want to cache data in serialized form, as
+it leads to much smaller sizes than Java serialization (and certainly than raw Java objects).
+
+## Garbage Collection Tuning
+
+JVM garbage collection can be a problem when you have large "churn" in terms of the RDDs
+stored by your program. (It is usually not a problem in programs that just read an RDD once
+and then run many operations on it.) When Java needs to evict old objects to make room for new ones, it will
+need to trace through all your Java objects and find the unused ones. The main point to remember here is
+that *the cost of garbage collection is proportional to the number of Java objects*, so using data
+structures with fewer objects (e.g. an array of `Int`s instead of a `LinkedList`) greatly lowers
+this cost. An even better method is to persist objects in serialized form, as described above: now
+there will be only *one* object (a byte array) per RDD partition. Before trying other
+techniques, the first thing to try if GC is a problem is to use [serialized caching](#serialized-rdd-storage).
+
+GC can also be a problem due to interference between your tasks' working memory (the
+amount of space needed to run the task) and the RDDs cached on your nodes. We will discuss how to control
+the space allocated to the RDD cache to mitigate this.
+
+**Measuring the Impact of GC**
+
+The first step in GC tuning is to collect statistics on how frequently garbage collection occurs and the amount of
+time spent GC. This can be done by adding `-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps` to your
+`SPARK_JAVA_OPTS` environment variable. Next time your Spark job is run, you will see messages printed in the worker's logs
+each time a garbage collection occurs. Note these logs will be on your cluster's worker nodes (in the `stdout` files in
+their work directories), *not* on your driver program.
+
+**Cache Size Tuning**
+
+One important configuration parameter for GC is the amount of memory that should be used for
+caching RDDs. By default, Spark uses 66% of the configured memory (`SPARK_MEM`) to cache RDDs. This means that
+ 33% of memory is available for any objects created during task execution.
+
+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,
+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.
+
+**Advanced GC Tuning**
+
+To further tune garbage collection, we first need to understand some basic information about memory management in the JVM:
+
+* Java Heap space is divided in to two regions Young and Old. The Young generation is meant to hold short-lived objects
+ while the Old generation is intended for objects with longer lifetimes.
+
+* The Young generation is further divided into three regions [Eden, Survivor1, Survivor2].
+
+* A simplified description of the garbage collection procedure: When Eden is full, a minor GC is run on Eden and objects
+ that are alive from Eden and Survivor1 are copied to Survivor2. The Survivor regions are swapped. If an object is old
+ enough or Survivor2 is full, it is moved to Old. Finally when Old is close to full, a full GC is invoked.
+
+The goal of GC tuning in Spark is to ensure that only long-lived RDDs are stored in the Old generation and that
+the Young generation is sufficiently sized to store short-lived objects. This will help avoid full GCs to collect
+temporary objects created during task execution. Some steps which may be useful are:
+
+* Check if there are too many garbage collections by collecting GC stats. If a full GC is invoked multiple times for
+ before a task completes, it means that there isn't enough memory available for executing tasks.
+
+* In the GC stats that are printed, if the OldGen is close to being full, reduce the amount of memory used for caching.
+ This can be done using the `spark.storage.memoryFraction` property. It is better to cache fewer objects than to slow
+ down task execution!
+
+* If there are too many minor collections but not many major GCs, allocating more memory for Eden would help. You
+ can set the size of the Eden to be an over-estimate of how much memory each task will need. If the size of Eden
+ is determined to be `E`, then you can set the size of the Young generation using the option `-Xmn=4/3*E`. (The scaling
+ up by 4/3 is to account for space used by survivor regions as well.)
+
+* As an example, if your task is reading data from HDFS, the amount of memory used by the task can be estimated using
+ the size of the data block read from HDFS. Note that the size of a decompressed block is often 2 or 3 times the
+ size of the block. So if we wish to have 3 or 4 tasks worth of working space, and the HDFS block size is 64 MB,
+ we can estimate size of Eden to be `4*3*64MB`.
+
+* Monitor how the frequency and time taken by garbage collection changes with the new settings.
+
+Our experience suggests that the effect of GC tuning depends on your application and the amount of memory available.
+There are [many more tuning options](http://www.oracle.com/technetwork/java/javase/gc-tuning-6-140523.html) described online,
+but at a high level, managing how frequently full GC takes place can help in reducing the overhead.
+
+# Other Considerations
+
+## Level of Parallelism
+
+Clusters will not be fully utilized unless you set the level of parallelism for each operation high
+enough. Spark automatically sets the number of "map" tasks to run on each file according to its size
+(though you can control it through optional parameters to `SparkContext.textFile`, etc), but for
+distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses a default value of 8.
+You can pass the level of parallelism as a second argument (see the
+[`spark.PairRDDFunctions`](api/core/index.html#spark.PairRDDFunctions) documentation),
+or set the system 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
+
+Sometimes, you will get an OutOfMemoryError not because your RDDs don't fit in memory, but because the
+working set of one of your tasks, such as one of the reduce tasks in `groupByKey`, was too large.
+Spark's shuffle operations (`sortByKey`, `groupByKey`, `reduceByKey`, `join`, etc) build a hash table
+within each task to perform the grouping, which can often be large. The simplest fix here is to
+*increase the level of parallelism*, so that each task's input set is smaller. Spark can efficiently
+support tasks as short as 200 ms, because it reuses one worker JVMs across all tasks and it has
+a low task launching cost, so you can safely increase the level of parallelism to more than the
+number of cores in your clusters.
+
+## Broadcasting Large Variables
+
+Using the [broadcast functionality](scala-programming-guide#broadcast-variables)
+available in `SparkContext` can greatly reduce the size of each serialized task, and the cost
+of launching a job over a cluster. If your tasks use any large object from the driver program
+inside of them (e.g. a static lookup table), consider turning it into a broadcast variable.
+Spark prints the serialized size of each task on the master, so you can look at that to
+decide whether your tasks are too large; in general tasks larger than about 20 KB are probably
+worth optimizing.
+
+# Summary
+
+This has been a short guide to point out the main concerns you should know about when tuning a
+Spark application -- most importantly, data serialization and memory tuning. For most programs,
+switching to Kryo serialization and persisting data in serialized form will solve most common
+performance issues. Feel free to ask on the
+[Spark mailing list](http://groups.google.com/group/spark-users) about other tuning best practices.
diff --git a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh
index 1f76e61b2f..50ecf83404 100644
--- a/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh
+++ b/ec2/deploy.generic/root/mesos-ec2/ec2-variables.sh
@@ -6,3 +6,4 @@ export MESOS_SLAVES="{{slave_list}}"
export MESOS_ZOO_LIST="{{zoo_list}}"
export MESOS_HDFS_DATA_DIRS="{{hdfs_data_dirs}}"
export MESOS_MAPRED_LOCAL_DIRS="{{mapred_local_dirs}}"
+export MESOS_SPARK_LOCAL_DIRS="{{spark_local_dirs}}"
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 931e4068de..6a3647b218 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -35,8 +35,7 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, EBSBlockDeviceType
# A static URL from which to figure out the latest Mesos EC2 AMI
-LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.5"
-LATEST_STANDALONE_AMI_URL = "https://s3.amazonaws.com/spark-standalone-amis/latest-spark"
+LATEST_AMI_URL = "https://s3.amazonaws.com/mesos-images/ids/latest-spark-0.6"
# Configure and parse our command-line arguments
@@ -59,12 +58,13 @@ def parse_args():
"WARNING: must be 64-bit; small instances won't work")
parser.add_option("-m", "--master-instance-type", default="",
help="Master instance type (leave empty for same as instance-type)")
- parser.add_option("-z", "--zone", default="us-east-1b",
+ parser.add_option("-r", "--region", default="us-east-1",
+ help="EC2 region zone to launch instances in")
+ parser.add_option("-z", "--zone", default="",
help="Availability zone to launch instances in")
parser.add_option("-a", "--ami", default="latest",
help="Amazon Machine Image ID to use, or 'latest' to use latest " +
- "availabe mesos AMI, 'standalone' for the latest available " +
- "standalone AMI (default: latest)")
+ "available AMI (default: latest)")
parser.add_option("-D", metavar="[ADDRESS:]PORT", dest="proxy_port",
help="Use SSH dynamic port forwarding to create a SOCKS proxy at " +
"the given local address (for use with login)")
@@ -191,19 +191,14 @@ def launch_cluster(conn, opts, cluster_name):
"group %s, %s or %s" % (master_group.name, slave_group.name, zoo_group.name))
sys.exit(1)
- if opts.ami in ["latest", "standalone"]:
-
- # Figure out the latest AMI from our static URL
- if opts.ami == "latest":
- url = LATEST_AMI_URL
- elif opts.ami == "standalone":
- url = LATEST_STANDALONE_AMI_URL
-
+ # Figure out the latest AMI from our static URL
+ if opts.ami == "latest":
try:
- opts.ami = urllib2.urlopen(url).read().strip()
+ opts.ami = urllib2.urlopen(LATEST_AMI_URL).read().strip()
print "Latest Spark AMI: " + opts.ami
except:
- print >> stderr, "Could not read " + url
+ print >> stderr, "Could not read " + LATEST_AMI_URL
+ sys.exit(1)
print "Launching instances..."
@@ -294,7 +289,7 @@ def launch_cluster(conn, opts, cluster_name):
# Get the EC2 instances in an existing cluster if available.
# Returns a tuple of lists of EC2 instance objects for the masters,
# slaves and zookeeper nodes (in that order).
-def get_existing_cluster(conn, opts, cluster_name):
+def get_existing_cluster(conn, opts, cluster_name, die_on_error=True):
print "Searching for existing cluster " + cluster_name + "..."
reservations = conn.get_all_instances()
master_nodes = []
@@ -310,9 +305,10 @@ def get_existing_cluster(conn, opts, cluster_name):
slave_nodes += res.instances
elif group_names == [cluster_name + "-zoo"]:
zoo_nodes += res.instances
- if master_nodes != [] and slave_nodes != []:
+ if any((master_nodes, slave_nodes, zoo_nodes)):
print ("Found %d master(s), %d slaves, %d ZooKeeper nodes" %
(len(master_nodes), len(slave_nodes), len(zoo_nodes)))
+ if (master_nodes != [] and slave_nodes != []) or not die_on_error:
return (master_nodes, slave_nodes, zoo_nodes)
else:
if master_nodes == [] and slave_nodes != []:
@@ -369,6 +365,7 @@ def get_num_disks(instance_type):
# From http://docs.amazonwebservices.com/AWSEC2/latest/UserGuide/index.html?InstanceStorage.html
disks_by_instance = {
"m1.small": 1,
+ "m1.medium": 1,
"m1.large": 2,
"m1.xlarge": 4,
"t1.micro": 1,
@@ -400,10 +397,12 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes):
num_disks = get_num_disks(opts.instance_type)
hdfs_data_dirs = "/mnt/ephemeral-hdfs/data"
mapred_local_dirs = "/mnt/hadoop/mrlocal"
+ spark_local_dirs = "/mnt/spark"
if num_disks > 1:
for i in range(2, num_disks + 1):
hdfs_data_dirs += ",/mnt%d/ephemeral-hdfs/data" % i
mapred_local_dirs += ",/mnt%d/hadoop/mrlocal" % i
+ spark_local_dirs += ",/mnt%d/spark" % i
if zoo_nodes != []:
zoo_list = '\n'.join([i.public_dns_name for i in zoo_nodes])
@@ -423,7 +422,8 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, zoo_nodes):
"zoo_list": zoo_list,
"cluster_url": cluster_url,
"hdfs_data_dirs": hdfs_data_dirs,
- "mapred_local_dirs": mapred_local_dirs
+ "mapred_local_dirs": mapred_local_dirs,
+ "spark_local_dirs": spark_local_dirs
}
# Create a temp directory in which we will place all the files to be
@@ -470,7 +470,7 @@ def ssh(host, opts, command):
def main():
(opts, action, cluster_name) = parse_args()
- conn = boto.connect_ec2()
+ conn = boto.ec2.connect_to_region(opts.region)
# Select an AZ at random if it was not specified.
if opts.zone == "":
@@ -492,7 +492,7 @@ def main():
"Destroy cluster " + cluster_name + " (y/N): ")
if response == "y":
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
- conn, opts, cluster_name)
+ conn, opts, cluster_name, die_on_error=False)
print "Terminating master..."
for inst in master_nodes:
inst.terminate()
@@ -527,7 +527,7 @@ def main():
"Stop cluster " + cluster_name + " (y/N): ")
if response == "y":
(master_nodes, slave_nodes, zoo_nodes) = get_existing_cluster(
- conn, opts, cluster_name)
+ conn, opts, cluster_name, die_on_error=False)
print "Stopping master..."
for inst in master_nodes:
if inst.state not in ["shutting-down", "terminated"]:
diff --git a/examples/src/main/java/spark/examples/JavaHdfsLR.java b/examples/src/main/java/spark/examples/JavaHdfsLR.java
index 71fc13fbce..29839d5668 100644
--- a/examples/src/main/java/spark/examples/JavaHdfsLR.java
+++ b/examples/src/main/java/spark/examples/JavaHdfsLR.java
@@ -1,6 +1,5 @@
package spark.examples;
-import scala.util.Random;
import spark.api.java.JavaRDD;
import spark.api.java.JavaSparkContext;
import spark.api.java.function.Function;
@@ -9,6 +8,7 @@ import spark.api.java.function.Function2;
import java.io.Serializable;
import java.util.Arrays;
import java.util.StringTokenizer;
+import java.util.Random;
public class JavaHdfsLR {
diff --git a/examples/src/main/java/spark/examples/JavaTC.java b/examples/src/main/java/spark/examples/JavaTC.java
index 25a465ec8e..e3bd881b8f 100644
--- a/examples/src/main/java/spark/examples/JavaTC.java
+++ b/examples/src/main/java/spark/examples/JavaTC.java
@@ -1,7 +1,6 @@
package spark.examples;
import scala.Tuple2;
-import scala.util.Random;
import spark.api.java.JavaPairRDD;
import spark.api.java.JavaSparkContext;
import spark.api.java.function.PairFunction;
@@ -9,6 +8,7 @@ import spark.api.java.function.PairFunction;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
+import java.util.Random;
import java.util.Set;
/**
diff --git a/examples/src/main/scala/spark/examples/BroadcastTest.scala b/examples/src/main/scala/spark/examples/BroadcastTest.scala
index 4a560131e3..230097c7db 100644
--- a/examples/src/main/scala/spark/examples/BroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/BroadcastTest.scala
@@ -5,7 +5,7 @@ import spark.SparkContext
object BroadcastTest {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: BroadcastTest <host> [<slices>] [numElem]")
+ System.err.println("Usage: BroadcastTest <master> [<slices>] [numElem]")
System.exit(1)
}
@@ -17,9 +17,13 @@ object BroadcastTest {
for (i <- 0 until arr1.length)
arr1(i) = i
- val barr1 = spark.broadcast(arr1)
- spark.parallelize(1 to 10, slices).foreach {
- i => println(barr1.value.size)
+ for (i <- 0 until 2) {
+ println("Iteration " + i)
+ println("===========")
+ val barr1 = spark.broadcast(arr1)
+ spark.parallelize(1 to 10, slices).foreach {
+ i => println(barr1.value.size)
+ }
}
System.exit(0)
diff --git a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
index 90941ddbf0..c89f3dac0c 100644
--- a/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
+++ b/examples/src/main/scala/spark/examples/ExceptionHandlingTest.scala
@@ -5,7 +5,7 @@ import spark.SparkContext
object ExceptionHandlingTest {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: ExceptionHandlingTest <host>")
+ System.err.println("Usage: ExceptionHandlingTest <master>")
System.exit(1)
}
diff --git a/examples/src/main/scala/spark/examples/GroupByTest.scala b/examples/src/main/scala/spark/examples/GroupByTest.scala
index 9d11692aeb..86dfba3a40 100644
--- a/examples/src/main/scala/spark/examples/GroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/GroupByTest.scala
@@ -7,7 +7,7 @@ import java.util.Random
object GroupByTest {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: GroupByTest <host> [numMappers] [numKVPairs] [KeySize] [numReducers]")
+ System.err.println("Usage: GroupByTest <master> [numMappers] [numKVPairs] [KeySize] [numReducers]")
System.exit(1)
}
diff --git a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
index 240376da90..83ae014e94 100644
--- a/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/spark/examples/MultiBroadcastTest.scala
@@ -5,7 +5,7 @@ import spark.SparkContext
object MultiBroadcastTest {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: BroadcastTest <host> [<slices>] [numElem]")
+ System.err.println("Usage: BroadcastTest <master> [<slices>] [numElem]")
System.exit(1)
}
diff --git a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
index c44b42c5b6..50b3a263b4 100644
--- a/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SimpleSkewedGroupByTest.scala
@@ -7,7 +7,7 @@ import java.util.Random
object SimpleSkewedGroupByTest {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: SimpleSkewedGroupByTest <host> " +
+ System.err.println("Usage: SimpleSkewedGroupByTest <master> " +
"[numMappers] [numKVPairs] [valSize] [numReducers] [ratio]")
System.exit(1)
}
diff --git a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
index 9fad7ab2e5..d2117a263e 100644
--- a/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
+++ b/examples/src/main/scala/spark/examples/SkewedGroupByTest.scala
@@ -7,7 +7,7 @@ import java.util.Random
object SkewedGroupByTest {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: GroupByTest <host> [numMappers] [numKVPairs] [KeySize] [numReducers]")
+ System.err.println("Usage: GroupByTest <master> [numMappers] [numKVPairs] [KeySize] [numReducers]")
System.exit(1)
}
diff --git a/examples/src/main/scala/spark/examples/SparkALS.scala b/examples/src/main/scala/spark/examples/SparkALS.scala
index 59d5154e08..fb28e2c932 100644
--- a/examples/src/main/scala/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/spark/examples/SparkALS.scala
@@ -107,7 +107,7 @@ object SparkALS {
host = host_
}
case _ => {
- System.err.println("Usage: SparkALS <M> <U> <F> <iters> <slices> <host>")
+ System.err.println("Usage: SparkALS <M> <U> <F> <iters> <slices> <master>")
System.exit(1)
}
}
diff --git a/examples/src/main/scala/spark/examples/SparkLR.scala b/examples/src/main/scala/spark/examples/SparkLR.scala
index 19123db738..aaaf062c8f 100644
--- a/examples/src/main/scala/spark/examples/SparkLR.scala
+++ b/examples/src/main/scala/spark/examples/SparkLR.scala
@@ -25,12 +25,12 @@ object SparkLR {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: SparkLR <host> [<slices>]")
+ System.err.println("Usage: SparkLR <master> [<slices>]")
System.exit(1)
}
val sc = new SparkContext(args(0), "SparkLR")
val numSlices = if (args.length > 1) args(1).toInt else 2
- val data = generateData
+ val points = sc.parallelize(generateData, numSlices).cache()
// Initialize w to a random value
var w = Vector(D, _ => 2 * rand.nextDouble - 1)
@@ -38,7 +38,7 @@ object SparkLR {
for (i <- 1 to ITERATIONS) {
println("On iteration " + i)
- val gradient = sc.parallelize(data, numSlices).map { p =>
+ val gradient = points.map { p =>
(1 / (1 + exp(-p.y * (w dot p.x))) - 1) * p.y * p.x
}.reduce(_ + _)
w -= gradient
diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala
index 751fef6ab0..2f226f1380 100644
--- a/examples/src/main/scala/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/spark/examples/SparkPi.scala
@@ -7,7 +7,7 @@ import SparkContext._
object SparkPi {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: SparkPi <host> [<slices>]")
+ System.err.println("Usage: SparkPi <master> [<slices>]")
System.exit(1)
}
val spark = new SparkContext(args(0), "SparkPi")
diff --git a/examples/src/main/scala/spark/examples/SparkTC.scala b/examples/src/main/scala/spark/examples/SparkTC.scala
index a6e4de4671..90bae011ad 100644
--- a/examples/src/main/scala/spark/examples/SparkTC.scala
+++ b/examples/src/main/scala/spark/examples/SparkTC.scala
@@ -26,7 +26,7 @@ object SparkTC {
def main(args: Array[String]) {
if (args.length == 0) {
- System.err.println("Usage: SparkTC <host> [<slices>]")
+ System.err.println("Usage: SparkTC <master> [<slices>]")
System.exit(1)
}
val spark = new SparkContext(args(0), "SparkTC")
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 358213fe64..688bb16a03 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -1,15 +1,23 @@
import sbt._
+import sbt.Classpaths.publishTask
import Keys._
import sbtassembly.Plugin._
import AssemblyKeys._
import twirl.sbt.TwirlPlugin._
+// For Sonatype publishing
+//import com.jsuereth.pgp.sbtplugin.PgpKeys._
object SparkBuild extends Build {
// Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or
- // "1.0.1" for Apache releases, or "0.20.2-cdh3u3" for Cloudera Hadoop.
+ // "1.0.3" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop.
val HADOOP_VERSION = "0.20.205.0"
+ val HADOOP_MAJOR_VERSION = "1"
- lazy val root = Project("root", file("."), settings = sharedSettings) aggregate(core, repl, examples, bagel, streaming)
+ // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2"
+ //val HADOOP_VERSION = "2.0.0-mr1-cdh4.1.1"
+ //val HADOOP_MAJOR_VERSION = "2"
+
+ lazy val root = Project("root", file("."), settings = rootSettings) aggregate(core, repl, examples, bagel, streaming)
lazy val core = Project("core", file("core"), settings = coreSettings)
@@ -21,16 +29,64 @@ object SparkBuild extends Build {
lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn (core)
+ // 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")
+
def sharedSettings = Defaults.defaultSettings ++ Seq(
organization := "org.spark-project",
- version := "0.6.0-SNAPSHOT",
- scalaVersion := "2.9.1",
+ version := "0.7.0-SNAPSHOT",
+ scalaVersion := "2.9.2",
scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue
unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath },
retrieveManaged := true,
transitiveClassifiers in Scope.GlobalScope := Seq("sources"),
testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))),
- publishTo <<= baseDirectory { base => Some(Resolver.file("Local", base / "target" / "maven" asFile)(Patterns(true, Resolver.mavenStyleBasePattern))) },
+
+ // 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/"),
+
+ publishMavenStyle := true,
+
+ //useGpg in Global := true,
+
+ pomExtra := (
+ <url>http://spark-project.org/</url>
+ <licenses>
+ <license>
+ <name>BSD License</name>
+ <url>https://github.com/mesos/spark/blob/master/LICENSE</url>
+ <distribution>repo</distribution>
+ </license>
+ </licenses>
+ <scm>
+ <connection>scm:git:git@github.com:mesos/spark.git</connection>
+ <url>scm:git:git@github.com:mesos/spark.git</url>
+ </scm>
+ <developers>
+ <developer>
+ <id>matei</id>
+ <name>Matei Zaharia</name>
+ <email>matei.zaharia@gmail.com</email>
+ <url>http://www.cs.berkeley.edu/~matei</url>
+ <organization>U.C. Berkeley Computer Science</organization>
+ <organizationUrl>http://www.cs.berkeley.edu/</organizationUrl>
+ </developer>
+ </developers>
+ ),
+
+/*
+ publishTo <<= version { (v: String) =>
+ val nexus = "https://oss.sonatype.org/"
+ if (v.trim.endsWith("SNAPSHOT"))
+ Some("sonatype-snapshots" at nexus + "content/repositories/snapshots")
+ else
+ Some("sonatype-staging" at nexus + "service/local/staging/deploy/maven2")
+ },
+
+*/
+
libraryDependencies ++= Seq(
"org.eclipse.jetty" % "jetty-server" % "7.5.3.v20111011",
"org.scalatest" %% "scalatest" % "1.6.1" % "test",
@@ -40,7 +96,15 @@ object SparkBuild extends Build {
parallelExecution := false,
/* 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) }
+ const(std.TaskExtra.constant(Nil)), aggregate = true, includeRoot = true) apply { _.join.map(_.flatten) },
+
+ otherResolvers := Seq(Resolver.file("dotM2", file(Path.userHome + "/.m2/repository"))),
+ publishLocalConfiguration in MavenCompile <<= (packagedArtifacts, deliverLocal, ivyLoggingLevel) map {
+ (arts, _, level) => new PublishConfiguration(None, "dotM2", arts, Seq(), level)
+ },
+ publishMavenStyle in MavenCompile := true,
+ publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal),
+ publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn
)
val slf4jVersion = "1.6.1"
@@ -50,9 +114,10 @@ object SparkBuild extends Build {
resolvers ++= Seq(
"Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/",
"JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/",
- "Cloudera Repository" at "http://repository.cloudera.com/artifactory/cloudera-repos/",
- "Spray Repository" at "http://repo.spray.cc/"
+ "Spray Repository" at "http://repo.spray.cc/",
+ "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/"
),
+
libraryDependencies ++= Seq(
"com.google.guava" % "guava" % "11.0.1",
"log4j" % "log4j" % "1.2.16",
@@ -63,20 +128,26 @@ object SparkBuild extends Build {
"asm" % "asm-all" % "3.3.1",
"com.google.protobuf" % "protobuf-java" % "2.4.1",
"de.javakaffee" % "kryo-serializers" % "0.9",
- "com.typesafe.akka" % "akka-actor" % "2.0.2",
- "com.typesafe.akka" % "akka-remote" % "2.0.2",
- "com.typesafe.akka" % "akka-slf4j" % "2.0.2",
+ "com.typesafe.akka" % "akka-actor" % "2.0.3",
+ "com.typesafe.akka" % "akka-remote" % "2.0.3",
+ "com.typesafe.akka" % "akka-slf4j" % "2.0.3",
"it.unimi.dsi" % "fastutil" % "6.4.4",
"colt" % "colt" % "1.2.0",
"cc.spray" % "spray-can" % "1.0-M2.1",
- "cc.spray" % "spray-server" % "1.0-M2.1"
- )
+ "cc.spray" % "spray-server" % "1.0-M2.1",
+ "org.apache.mesos" % "mesos" % "0.9.0-incubating"
+ ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq,
+ unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") }
) ++ assemblySettings ++ extraAssemblySettings ++ Twirl.settings
+ def rootSettings = sharedSettings ++ Seq(
+ publish := {}
+ )
+
def replSettings = sharedSettings ++ Seq(
name := "spark-repl",
libraryDependencies <+= scalaVersion("org.scala-lang" % "scala-compiler" % _)
- ) ++ assemblySettings ++ extraAssemblySettings
+ )
def examplesSettings = sharedSettings ++ Seq(
name := "spark-examples"
@@ -89,11 +160,10 @@ object SparkBuild extends Build {
) ++ assemblySettings ++ extraAssemblySettings
def extraAssemblySettings() = Seq(test in assembly := {}) ++ Seq(
- mergeStrategy in assembly := {
- case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
+ mergeStrategy in assembly := {
+ case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard
case "reference.conf" => MergeStrategy.concat
case _ => MergeStrategy.first
}
- )
-
+ )
}
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 896fa4834f..4d0e696a11 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -11,3 +11,8 @@ addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.1.0-RC1")
addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.0.0")
addSbtPlugin("cc.spray" %% "sbt-twirl" % "0.5.2")
+
+// For Sonatype publishing
+//resolvers += Resolver.url("sbt-plugin-releases", new URL("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases/"))(Resolver.ivyStylePatterns)
+
+//addSbtPlugin("com.jsuereth" % "xsbt-gpg-plugin" % "0.6")
diff --git a/repl/src/main/scala/spark/repl/SparkILoop.scala b/repl/src/main/scala/spark/repl/SparkILoop.scala
index faece8baa4..22bcb4be8a 100644
--- a/repl/src/main/scala/spark/repl/SparkILoop.scala
+++ b/repl/src/main/scala/spark/repl/SparkILoop.scala
@@ -200,7 +200,7 @@ class SparkILoop(in0: Option[BufferedReader], val out: PrintWriter, val master:
____ __
/ __/__ ___ _____/ /__
_\ \/ _ \/ _ `/ __/ '_/
- /___/ .__/\_,_/_/ /_/\_\ version 0.6.0-SNAPSHOT
+ /___/ .__/\_,_/_/ /_/\_\ version 0.7.0
/_/
""")
import Properties._
diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties
index 02fe16866e..4c99e450bc 100644
--- a/repl/src/test/resources/log4j.properties
+++ b/repl/src/test/resources/log4j.properties
@@ -1,8 +1,10 @@
# Set everything to be logged to the console
-log4j.rootCategory=WARN, 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.SSS} %p %c{1}: %m%n
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=false
+log4j.appender.file.file=spark-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/repl/src/test/scala/spark/repl/ReplSuite.scala b/repl/src/test/scala/spark/repl/ReplSuite.scala
index 15ebf0c9b8..db78d06d4f 100644
--- a/repl/src/test/scala/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/spark/repl/ReplSuite.scala
@@ -7,6 +7,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConversions._
import org.scalatest.FunSuite
+import com.google.common.io.Files
class ReplSuite extends FunSuite {
def runInterpreter(master: String, input: String): String = {
@@ -29,6 +30,8 @@ class ReplSuite extends FunSuite {
spark.repl.Main.interp = null
if (interp.sparkContext != null)
interp.sparkContext.stop()
+ // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
+ System.clearProperty("spark.master.port")
return out.toString
}
@@ -118,7 +121,27 @@ class ReplSuite extends FunSuite {
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") {
+ 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))
+ assertDoesNotContain("error:", output)
+ assertDoesNotContain("Exception", output)
+ assertContains("res0: Long = 3", output)
+ assertContains("res1: Long = 3", output)
+ assertContains("res2: Long = 3", output)
+ }
+
if (System.getenv("MESOS_NATIVE_LIBRARY") != null) {
test ("running on Mesos") {
val output = runInterpreter("localquiet", """
diff --git a/run b/run
index e3e98f4280..a363599cf0 100755
--- a/run
+++ b/run
@@ -1,6 +1,6 @@
#!/bin/bash
-SCALA_VERSION=2.9.1
+SCALA_VERSION=2.9.2
# Figure out where the Scala framework is installed
FWDIR="$(cd `dirname $0`; pwd)"
@@ -13,16 +13,33 @@ if [ -e $FWDIR/conf/spark-env.sh ] ; then
. $FWDIR/conf/spark-env.sh
fi
-# Check that SCALA_HOME has been specified
-if [ -z "$SCALA_HOME" ]; then
- echo "SCALA_HOME is not set" >&2
- exit 1
-fi
-
-# If the user specifies a Mesos JAR, put it before our included one on the classpath
-MESOS_CLASSPATH=""
-if [ -z "$MESOS_JAR" ] ; then
- MESOS_CLASSPATH="$MESOS_JAR"
+if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
+ if [ `command -v scala` ]; then
+ RUNNER="scala"
+ else
+ if [ -z "$SCALA_HOME" ]; then
+ echo "SCALA_HOME is not set" >&2
+ exit 1
+ fi
+ RUNNER="${SCALA_HOME}/bin/scala"
+ fi
+else
+ if [ `command -v java` ]; then
+ RUNNER="java"
+ else
+ if [ -z "$JAVA_HOME" ]; then
+ echo "JAVA_HOME is not set" >&2
+ exit 1
+ fi
+ RUNNER="${JAVA_HOME}/bin/java"
+ fi
+ if [ -z "$SCALA_LIBRARY_PATH" ]; then
+ if [ -z "$SCALA_HOME" ]; then
+ echo "SCALA_HOME is not set" >&2
+ exit 1
+ fi
+ SCALA_LIBRARY_PATH="$SCALA_HOME/lib"
+ fi
fi
# Figure out how much memory to use per executor and set it as an environment
@@ -50,16 +67,15 @@ STREAMING_DIR="$FWDIR/streaming"
# Build up classpath
CLASSPATH="$SPARK_CLASSPATH"
-CLASSPATH+=":$MESOS_CLASSPATH"
CLASSPATH+=":$FWDIR/conf"
CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/classes"
+if [ -n "$SPARK_TESTING" ] ; then
+ CLASSPATH+=":$CORE_DIR/target/scala-$SCALA_VERSION/test-classes"
+fi
CLASSPATH+=":$CORE_DIR/src/main/resources"
CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes"
CLASSPATH+=":$STREAMING_DIR/target/scala-$SCALA_VERSION/classes"
-for jar in `find $CORE_DIR/lib -name '*jar'`; do
- CLASSPATH+=":$jar"
-done
for jar in `find $FWDIR/lib_managed/jars -name '*jar'`; do
CLASSPATH+=":$jar"
done
@@ -79,17 +95,11 @@ export CLASSPATH # Needed for spark-shell
# the Spark shell, the wrapper is necessary to properly reset the terminal
# when we exit, so we allow it to set a variable to launch with scala.
if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
- RUNNER="${SCALA_HOME}/bin/scala"
EXTRA_ARGS="" # Java options will be passed to scala as JAVA_OPTS
else
- CLASSPATH+=":$SCALA_HOME/lib/scala-library.jar"
- CLASSPATH+=":$SCALA_HOME/lib/scala-compiler.jar"
- CLASSPATH+=":$SCALA_HOME/lib/jline.jar"
- if [ -n "$JAVA_HOME" ]; then
- RUNNER="${JAVA_HOME}/bin/java"
- else
- RUNNER=java
- fi
+ CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-library.jar"
+ CLASSPATH+=":$SCALA_LIBRARY_PATH/scala-compiler.jar"
+ CLASSPATH+=":$SCALA_LIBRARY_PATH/jline.jar"
# The JVM doesn't read JAVA_OPTS by default so we need to pass it in
EXTRA_ARGS="$JAVA_OPTS"
fi
diff --git a/run.cmd b/run.cmd
new file mode 100644
index 0000000000..cc5605f8a9
--- /dev/null
+++ b/run.cmd
@@ -0,0 +1,2 @@
+@echo off
+cmd /V /E /C %~dp0run2.cmd %*
diff --git a/run2.cmd b/run2.cmd
new file mode 100644
index 0000000000..097718b526
--- /dev/null
+++ b/run2.cmd
@@ -0,0 +1,67 @@
+@echo off
+
+set SCALA_VERSION=2.9.1
+
+rem Figure out where the Spark framework is installed
+set FWDIR=%~dp0
+
+rem Export this as SPARK_HOME
+set SPARK_HOME=%FWDIR%
+
+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 Check that SCALA_HOME has been specified
+if not "x%SCALA_HOME%"=="x" goto scala_exists
+ echo "SCALA_HOME is not set"
+ goto exit
+:scala_exists
+
+rem If the user specifies a Mesos JAR, put it before our included one on the classpath
+set MESOS_CLASSPATH=
+if not "x%MESOS_JAR%"=="x" set MESOS_CLASSPATH=%MESOS_JAR%
+
+rem Figure out how much memory to use per executor and set it as an environment
+rem variable so that our process sees it and can report it to Mesos
+if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m
+
+rem Set JAVA_OPTS to be able to load native libraries and to set heap size
+set JAVA_OPTS=%SPARK_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM%
+rem Load extra JAVA_OPTS from conf/java-opts, if it exists
+if exist "%FWDIR%conf\java-opts.cmd" call "%FWDIR%conf\java-opts.cmd"
+
+set CORE_DIR=%FWDIR%core
+set REPL_DIR=%FWDIR%repl
+set EXAMPLES_DIR=%FWDIR%examples
+set BAGEL_DIR=%FWDIR%bagel
+
+rem Build up classpath
+set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes
+set CLASSPATH=%CLASSPATH%;%CORE_DIR%\target\scala-%SCALA_VERSION%\test-classes;%CORE_DIR%\src\main\resources
+set CLASSPATH=%CLASSPATH%;%REPL_DIR%\target\scala-%SCALA_VERSION%\classes;%EXAMPLES_DIR%\target\scala-%SCALA_VERSION%\classes
+for /R "%FWDIR%\lib_managed\jars" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j
+for /R "%FWDIR%\lib_managed\bundles" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j
+for /R "%REPL_DIR%\lib" %%j in (*.jar) do set CLASSPATH=!CLASSPATH!;%%j
+set CLASSPATH=%CLASSPATH%;%BAGEL_DIR%\target\scala-%SCALA_VERSION%\classes
+
+rem Figure out whether to run our class with java or with the scala launcher.
+rem In most cases, we'd prefer to execute our process with java because scala
+rem creates a shell script as the parent of its Java process, which makes it
+rem hard to kill the child with stuff like Process.destroy(). However, for
+rem the Spark shell, the wrapper is necessary to properly reset the terminal
+rem when we exit, so we allow it to set a variable to launch with scala.
+if "%SPARK_LAUNCH_WITH_SCALA%" NEQ 1 goto java_runner
+ set RUNNER=%SCALA_HOME%\bin\scala
+ # Java options will be passed to scala as JAVA_OPTS
+ set EXTRA_ARGS=
+ goto run_spark
+:java_runner
+ set CLASSPATH=%CLASSPATH%;%SCALA_HOME%\lib\scala-library.jar;%SCALA_HOME%\lib\scala-compiler.jar;%SCALA_HOME%\lib\jline.jar
+ set RUNNER=java
+ if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java
+ rem The JVM doesn't read JAVA_OPTS by default so we need to pass it in
+ set EXTRA_ARGS=%JAVA_OPTS%
+:run_spark
+
+%RUNNER% -cp "%CLASSPATH%" %EXTRA_ARGS% %*
+:exit
diff --git a/sbt/sbt b/sbt/sbt
index fab9967286..a3055c13c1 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -4,4 +4,5 @@ if [ "$MESOS_HOME" != "" ]; then
EXTRA_ARGS="-Djava.library.path=$MESOS_HOME/lib/java"
fi
export SPARK_HOME=$(cd "$(dirname $0)/.."; pwd)
+export SPARK_TESTING=1 # To put test classes on classpath
java -Xmx1200M -XX:MaxPermSize=200m $EXTRA_ARGS -jar $SPARK_HOME/sbt/sbt-launch-*.jar "$@"
diff --git a/sbt/sbt.cmd b/sbt/sbt.cmd
new file mode 100644
index 0000000000..6b289ab447
--- /dev/null
+++ b/sbt/sbt.cmd
@@ -0,0 +1,5 @@
+@echo off
+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 %EXTRA_ARGS% -jar %SPARK_HOME%\sbt\sbt-launch-*.jar "%*"
diff --git a/spark-shell.cmd b/spark-shell.cmd
new file mode 100644
index 0000000000..34697d52d7
--- /dev/null
+++ b/spark-shell.cmd
@@ -0,0 +1,4 @@
+@echo off
+set FWDIR=%~dp0
+set SPARK_LAUNCH_WITH_SCALA=1
+cmd /V /E /C %FWDIR%run2.cmd spark.repl.Main %*
diff --git a/streaming/src/main/scala/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
index ebff9bdb51..83a43d15cb 100644
--- a/streaming/src/main/scala/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/spark/streaming/Checkpoint.scala
@@ -10,7 +10,7 @@ import java.io.{InputStream, ObjectStreamClass, ObjectInputStream, ObjectOutputS
class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Serializable {
val master = ssc.sc.master
- val framework = ssc.sc.frameworkName
+ val framework = ssc.sc.jobName
val sparkHome = ssc.sc.sparkHome
val jars = ssc.sc.jars
val graph = ssc.graph
diff --git a/streaming/src/main/scala/spark/streaming/CoGroupedDStream.scala b/streaming/src/main/scala/spark/streaming/CoGroupedDStream.scala
index 5522e2ee21..61d088eddb 100644
--- a/streaming/src/main/scala/spark/streaming/CoGroupedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/CoGroupedDStream.scala
@@ -1,6 +1,7 @@
package spark.streaming
-import spark.{CoGroupedRDD, RDD, Partitioner}
+import spark.{RDD, Partitioner}
+import spark.rdd.CoGroupedRDD
class CoGroupedDStream[K : ClassManifest](
parents: Seq[DStream[(_, _)]],
diff --git a/streaming/src/main/scala/spark/streaming/DStream.scala b/streaming/src/main/scala/spark/streaming/DStream.scala
index 4bc063719c..12d7ba97ea 100644
--- a/streaming/src/main/scala/spark/streaming/DStream.scala
+++ b/streaming/src/main/scala/spark/streaming/DStream.scala
@@ -4,6 +4,7 @@ import spark.streaming.StreamingContext._
import spark._
import spark.SparkContext._
+import spark.rdd._
import spark.storage.StorageLevel
import scala.collection.mutable.ArrayBuffer
@@ -82,7 +83,7 @@ extends Serializable with Logging {
// Set caching level for the RDDs created by this DStream
def persist(newLevel: StorageLevel): DStream[T] = persist(newLevel, StorageLevel.NONE, null)
- def persist(): DStream[T] = persist(StorageLevel.MEMORY_ONLY_DESER)
+ def persist(): DStream[T] = persist(StorageLevel.MEMORY_ONLY)
// Turn on the default caching level for this RDD
def cache(): DStream[T] = persist()
diff --git a/streaming/src/main/scala/spark/streaming/FileInputDStream.scala b/streaming/src/main/scala/spark/streaming/FileInputDStream.scala
index 78537b8794..537ec88047 100644
--- a/streaming/src/main/scala/spark/streaming/FileInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/FileInputDStream.scala
@@ -1,7 +1,7 @@
package spark.streaming
import spark.RDD
-import spark.UnionRDD
+import spark.rdd.UnionRDD
import org.apache.hadoop.fs.{FileSystem, Path, PathFilter}
import org.apache.hadoop.conf.Configuration
diff --git a/streaming/src/main/scala/spark/streaming/NetworkInputDStream.scala b/streaming/src/main/scala/spark/streaming/NetworkInputDStream.scala
index 5669d7fedf..f3f4c3ab13 100644
--- a/streaming/src/main/scala/spark/streaming/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/NetworkInputDStream.scala
@@ -1,6 +1,9 @@
package spark.streaming
-import spark.{Logging, SparkEnv, RDD, BlockRDD}
+import scala.collection.mutable.ArrayBuffer
+
+import spark.{Logging, SparkEnv, RDD}
+import spark.rdd.BlockRDD
import spark.storage.StorageLevel
import java.nio.ByteBuffer
@@ -107,7 +110,8 @@ abstract class NetworkReceiver[T: ClassManifest](streamId: Int) extends Serializ
* This method pushes a block (as iterator of values) into the block manager.
*/
protected def pushBlock(blockId: String, iterator: Iterator[T], level: StorageLevel) {
- env.blockManager.put(blockId, iterator, level)
+ val buffer = new ArrayBuffer[T] ++ iterator
+ env.blockManager.put(blockId, buffer.asInstanceOf[ArrayBuffer[Any]], level)
actor ! ReportBlock(blockId)
}
diff --git a/streaming/src/main/scala/spark/streaming/QueueInputDStream.scala b/streaming/src/main/scala/spark/streaming/QueueInputDStream.scala
index b794159b09..bb86e51932 100644
--- a/streaming/src/main/scala/spark/streaming/QueueInputDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/QueueInputDStream.scala
@@ -1,7 +1,7 @@
package spark.streaming
import spark.RDD
-import spark.UnionRDD
+import spark.rdd.UnionRDD
import scala.collection.mutable.Queue
import scala.collection.mutable.ArrayBuffer
diff --git a/streaming/src/main/scala/spark/streaming/ReducedWindowedDStream.scala b/streaming/src/main/scala/spark/streaming/ReducedWindowedDStream.scala
index fcf57aced7..1c57d5f855 100644
--- a/streaming/src/main/scala/spark/streaming/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/ReducedWindowedDStream.scala
@@ -3,8 +3,8 @@ package spark.streaming
import spark.streaming.StreamingContext._
import spark.RDD
-import spark.UnionRDD
-import spark.CoGroupedRDD
+import spark.rdd.UnionRDD
+import spark.rdd.CoGroupedRDD
import spark.Partitioner
import spark.SparkContext._
import spark.storage.StorageLevel
diff --git a/streaming/src/main/scala/spark/streaming/StateDStream.scala b/streaming/src/main/scala/spark/streaming/StateDStream.scala
index 3ba8fb45fb..086752ac55 100644
--- a/streaming/src/main/scala/spark/streaming/StateDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/StateDStream.scala
@@ -1,9 +1,9 @@
package spark.streaming
import spark.RDD
-import spark.BlockRDD
+import spark.rdd.BlockRDD
import spark.Partitioner
-import spark.MapPartitionsRDD
+import spark.rdd.MapPartitionsRDD
import spark.SparkContext._
import spark.storage.StorageLevel
diff --git a/streaming/src/main/scala/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
index e124b8cfa0..7c7b3afe47 100644
--- a/streaming/src/main/scala/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/spark/streaming/StreamingContext.scala
@@ -89,7 +89,7 @@ class StreamingContext (
def networkTextStream(
hostname: String,
port: Int,
- storageLevel: StorageLevel = StorageLevel.DISK_AND_MEMORY_2
+ storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[String] = {
networkStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel)
}
@@ -108,7 +108,7 @@ class StreamingContext (
def rawNetworkStream[T: ClassManifest](
hostname: String,
port: Int,
- storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_2
+ storageLevel: StorageLevel = StorageLevel.MEMORY_ONLY_SER_2
): DStream[T] = {
val inputStream = new RawInputDStream[T](this, hostname, port, storageLevel)
graph.addInputStream(inputStream)
diff --git a/streaming/src/main/scala/spark/streaming/WindowedDStream.scala b/streaming/src/main/scala/spark/streaming/WindowedDStream.scala
index b90e22351b..ce89a3f99b 100644
--- a/streaming/src/main/scala/spark/streaming/WindowedDStream.scala
+++ b/streaming/src/main/scala/spark/streaming/WindowedDStream.scala
@@ -1,7 +1,7 @@
package spark.streaming
import spark.RDD
-import spark.UnionRDD
+import spark.rdd.UnionRDD
class WindowedDStream[T: ClassManifest](
diff --git a/streaming/src/main/scala/spark/streaming/examples/TopKWordCountRaw.scala b/streaming/src/main/scala/spark/streaming/examples/TopKWordCountRaw.scala
index 9d1b0b9eb4..57fd10f0a5 100644
--- a/streaming/src/main/scala/spark/streaming/examples/TopKWordCountRaw.scala
+++ b/streaming/src/main/scala/spark/streaming/examples/TopKWordCountRaw.scala
@@ -41,7 +41,7 @@ object TopKWordCountRaw {
val windowedCounts = union.mapPartitions(splitAndCountPartitions)
.reduceByKeyAndWindow(add _, subtract _, Seconds(30), Milliseconds(batchMs), reduces)
- windowedCounts.persist(StorageLevel.MEMORY_ONLY_DESER, StorageLevel.MEMORY_ONLY_DESER_2,
+ windowedCounts.persist(StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY_2,
Milliseconds(chkptMs))
//windowedCounts.print() // TODO: something else?
diff --git a/streaming/src/main/scala/spark/streaming/examples/WordCount2.scala b/streaming/src/main/scala/spark/streaming/examples/WordCount2.scala
index 8390f4af94..0d2e62b955 100644
--- a/streaming/src/main/scala/spark/streaming/examples/WordCount2.scala
+++ b/streaming/src/main/scala/spark/streaming/examples/WordCount2.scala
@@ -100,8 +100,8 @@ object WordCount2 {
val windowedCounts = sentences
.mapPartitions(splitAndCountPartitions)
.reduceByKeyAndWindow(add _, subtract _, Seconds(30), batchDuration, reduceTasks.toInt)
- windowedCounts.persist(StorageLevel.MEMORY_ONLY_DESER,
- StorageLevel.MEMORY_ONLY_DESER_2,
+ windowedCounts.persist(StorageLevel.MEMORY_ONLY,
+ StorageLevel.MEMORY_ONLY_2,
//new StorageLevel(false, true, true, 3),
Milliseconds(chkptMillis.toLong))
windowedCounts.foreachRDD(r => println("Element count: " + r.count()))
diff --git a/streaming/src/main/scala/spark/streaming/examples/WordCountRaw.scala b/streaming/src/main/scala/spark/streaming/examples/WordCountRaw.scala
index d8a0664d7d..abfd12890f 100644
--- a/streaming/src/main/scala/spark/streaming/examples/WordCountRaw.scala
+++ b/streaming/src/main/scala/spark/streaming/examples/WordCountRaw.scala
@@ -41,7 +41,7 @@ object WordCountRaw {
val windowedCounts = union.mapPartitions(splitAndCountPartitions)
.reduceByKeyAndWindow(add _, subtract _, Seconds(30), Milliseconds(batchMs), reduces)
- windowedCounts.persist(StorageLevel.MEMORY_ONLY_DESER, StorageLevel.MEMORY_ONLY_DESER_2,
+ windowedCounts.persist(StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY_2,
Milliseconds(chkptMs))
//windowedCounts.print() // TODO: something else?
windowedCounts.foreachRDD(r => println("Element count: " + r.count()))
diff --git a/streaming/src/main/scala/spark/streaming/examples/WordMax2.scala b/streaming/src/main/scala/spark/streaming/examples/WordMax2.scala
index fc7567322b..9d44da2b11 100644
--- a/streaming/src/main/scala/spark/streaming/examples/WordMax2.scala
+++ b/streaming/src/main/scala/spark/streaming/examples/WordMax2.scala
@@ -57,10 +57,10 @@ object WordMax2 {
val windowedCounts = sentences
.mapPartitions(splitAndCountPartitions)
.reduceByKey(add _, reduceTasks.toInt)
- .persist(StorageLevel.MEMORY_ONLY_DESER, StorageLevel.MEMORY_ONLY_DESER_2,
+ .persist(StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY_2,
Milliseconds(chkptMillis.toLong))
.reduceByKeyAndWindow(max _, Seconds(10), batchDuration, reduceTasks.toInt)
- //.persist(StorageLevel.MEMORY_ONLY_DESER, StorageLevel.MEMORY_ONLY_DESER_2,
+ //.persist(StorageLevel.MEMORY_ONLY, StorageLevel.MEMORY_ONLY_2,
// Milliseconds(chkptMillis.toLong))
windowedCounts.foreachRDD(r => println("Element count: " + r.count()))
diff --git a/streaming/src/main/scala/spark/streaming/util/TestStreamReceiver3.scala b/streaming/src/main/scala/spark/streaming/util/TestStreamReceiver3.scala
index d00ae9cbca..80ad924dd8 100644
--- a/streaming/src/main/scala/spark/streaming/util/TestStreamReceiver3.scala
+++ b/streaming/src/main/scala/spark/streaming/util/TestStreamReceiver3.scala
@@ -171,11 +171,11 @@ extends Thread with Logging {
logInfo("Pushing block")
val startTime = System.currentTimeMillis
- val bytes = blockManager.dataSerialize(block.data.toIterator)
+ val bytes = blockManager.dataSerialize("rdd_", block.data.toIterator) // TODO: Will this be an RDD block?
val finishTime = System.currentTimeMillis
logInfo(block + " serialization delay is " + (finishTime - startTime) / 1000.0 + " s")
- blockManager.putBytes(block.id.toString, bytes, StorageLevel.DISK_AND_MEMORY_2)
+ blockManager.putBytes(block.id.toString, bytes, StorageLevel.MEMORY_AND_DISK_SER_2)
/*blockManager.putBytes(block.id.toString, bytes, StorageLevel.DISK_AND_MEMORY_DESER_2)*/
/*blockManager.put(block.id.toString, block.data.toIterator, StorageLevel.DISK_AND_MEMORY_DESER)*/
/*blockManager.put(block.id.toString, block.data.toIterator, StorageLevel.DISK_AND_MEMORY)*/
diff --git a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
index a3f213ebd0..f81ab2607f 100644
--- a/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/spark/streaming/InputStreamsSuite.scala
@@ -18,7 +18,7 @@ class InputStreamsSuite extends TestSuiteBase {
val ssc = new StreamingContext(master, framework)
ssc.setBatchDuration(batchDuration)
- val networkStream = ssc.networkTextStream("localhost", serverPort, StorageLevel.DISK_AND_MEMORY)
+ val networkStream = ssc.networkTextStream("localhost", serverPort, StorageLevel.MEMORY_AND_DISK)
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String ]]
val outputStream = new TestOutputStream(networkStream, outputBuffer)
ssc.registerOutputStream(outputStream)