aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorhaitao.yao <yao.erix@gmail.com>2013-02-19 11:22:24 +0800
committerhaitao.yao <yao.erix@gmail.com>2013-02-19 11:22:24 +0800
commit7c129388fbdc90cb6abb99470545dba8a2e90adf (patch)
treee6801d75863f0309fcc7f0e5af92d1da73f27cbd /core
parent858784459f27da6b969022339dcda4cb9970de1b (diff)
parent7151e1e4c8f4f764c54047ef82b988f887a0b9c7 (diff)
downloadspark-7c129388fbdc90cb6abb99470545dba8a2e90adf.tar.gz
spark-7c129388fbdc90cb6abb99470545dba8a2e90adf.tar.bz2
spark-7c129388fbdc90cb6abb99470545dba8a2e90adf.zip
Merge branch 'mesos'
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/spark/CacheManager.scala4
-rw-r--r--core/src/main/scala/spark/DoubleRDDFunctions.scala4
-rw-r--r--core/src/main/scala/spark/PairRDDFunctions.scala62
-rw-r--r--core/src/main/scala/spark/Partition.scala (renamed from core/src/main/scala/spark/Split.scala)2
-rw-r--r--core/src/main/scala/spark/RDD.scala76
-rw-r--r--core/src/main/scala/spark/RDDCheckpointData.scala12
-rw-r--r--core/src/main/scala/spark/SparkContext.scala28
-rw-r--r--core/src/main/scala/spark/api/java/JavaDoubleRDD.scala6
-rw-r--r--core/src/main/scala/spark/api/java/JavaPairRDD.scala44
-rw-r--r--core/src/main/scala/spark/api/java/JavaRDD.scala6
-rw-r--r--core/src/main/scala/spark/api/java/JavaRDDLike.scala12
-rw-r--r--core/src/main/scala/spark/api/java/JavaSparkContext.scala22
-rw-r--r--core/src/main/scala/spark/api/python/PythonPartitioner.scala2
-rw-r--r--core/src/main/scala/spark/api/python/PythonRDD.scala10
-rw-r--r--core/src/main/scala/spark/deploy/ApplicationDescription.scala (renamed from core/src/main/scala/spark/deploy/JobDescription.scala)4
-rw-r--r--core/src/main/scala/spark/deploy/DeployMessage.scala19
-rw-r--r--core/src/main/scala/spark/deploy/JsonProtocol.scala18
-rw-r--r--core/src/main/scala/spark/deploy/client/Client.scala22
-rw-r--r--core/src/main/scala/spark/deploy/client/ClientListener.scala2
-rw-r--r--core/src/main/scala/spark/deploy/client/TestClient.scala6
-rw-r--r--core/src/main/scala/spark/deploy/master/ApplicationInfo.scala (renamed from core/src/main/scala/spark/deploy/master/JobInfo.scala)10
-rw-r--r--core/src/main/scala/spark/deploy/master/ApplicationState.scala11
-rw-r--r--core/src/main/scala/spark/deploy/master/ExecutorInfo.scala4
-rw-r--r--core/src/main/scala/spark/deploy/master/JobState.scala9
-rw-r--r--core/src/main/scala/spark/deploy/master/Master.scala174
-rw-r--r--core/src/main/scala/spark/deploy/master/MasterWebUI.scala22
-rw-r--r--core/src/main/scala/spark/deploy/master/WorkerInfo.scala4
-rw-r--r--core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala26
-rw-r--r--core/src/main/scala/spark/deploy/worker/Worker.scala20
-rw-r--r--core/src/main/scala/spark/deploy/worker/WorkerArguments.scala2
-rw-r--r--core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala4
-rw-r--r--core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala5
-rw-r--r--core/src/main/scala/spark/partial/ApproximateActionListener.scala2
-rw-r--r--core/src/main/scala/spark/rdd/BlockRDD.scala16
-rw-r--r--core/src/main/scala/spark/rdd/CartesianRDD.scala36
-rw-r--r--core/src/main/scala/spark/rdd/CheckpointRDD.scala20
-rw-r--r--core/src/main/scala/spark/rdd/CoGroupedRDD.scala26
-rw-r--r--core/src/main/scala/spark/rdd/CoalescedRDD.scala24
-rw-r--r--core/src/main/scala/spark/rdd/FilteredRDD.scala6
-rw-r--r--core/src/main/scala/spark/rdd/FlatMappedRDD.scala6
-rw-r--r--core/src/main/scala/spark/rdd/GlommedRDD.scala6
-rw-r--r--core/src/main/scala/spark/rdd/HadoopRDD.scala20
-rw-r--r--core/src/main/scala/spark/rdd/MapPartitionsRDD.scala8
-rw-r--r--core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala (renamed from core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala)12
-rw-r--r--core/src/main/scala/spark/rdd/MappedRDD.scala6
-rw-r--r--core/src/main/scala/spark/rdd/NewHadoopRDD.scala20
-rw-r--r--core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala (renamed from core/src/main/scala/spark/ParallelCollection.scala)31
-rw-r--r--core/src/main/scala/spark/rdd/PartitionPruningRDD.scala16
-rw-r--r--core/src/main/scala/spark/rdd/PipedRDD.scala6
-rw-r--r--core/src/main/scala/spark/rdd/SampledRDD.scala16
-rw-r--r--core/src/main/scala/spark/rdd/ShuffledRDD.scala10
-rw-r--r--core/src/main/scala/spark/rdd/UnionRDD.scala30
-rw-r--r--core/src/main/scala/spark/rdd/ZippedRDD.scala32
-rw-r--r--core/src/main/scala/spark/scheduler/DAGScheduler.scala14
-rw-r--r--core/src/main/scala/spark/scheduler/ResultTask.scala6
-rw-r--r--core/src/main/scala/spark/scheduler/ShuffleMapTask.scala6
-rw-r--r--core/src/main/scala/spark/scheduler/Stage.scala2
-rw-r--r--core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala15
-rw-r--r--core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala4
-rw-r--r--core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala4
-rw-r--r--core/src/main/scala/spark/storage/BlockManager.scala2
-rw-r--r--core/src/main/scala/spark/storage/StorageUtils.scala2
-rw-r--r--core/src/main/twirl/spark/deploy/master/app_details.scala.html40
-rw-r--r--core/src/main/twirl/spark/deploy/master/app_row.scala.html20
-rw-r--r--core/src/main/twirl/spark/deploy/master/app_table.scala.html (renamed from core/src/main/twirl/spark/deploy/master/job_table.scala.html)8
-rw-r--r--core/src/main/twirl/spark/deploy/master/executor_row.scala.html6
-rw-r--r--core/src/main/twirl/spark/deploy/master/index.scala.html16
-rw-r--r--core/src/main/twirl/spark/deploy/master/job_details.scala.html40
-rw-r--r--core/src/main/twirl/spark/deploy/master/job_row.scala.html20
-rw-r--r--core/src/main/twirl/spark/deploy/worker/executor_row.scala.html10
-rw-r--r--core/src/test/scala/spark/CheckpointSuite.scala112
-rw-r--r--core/src/test/scala/spark/RDDSuite.scala13
-rw-r--r--core/src/test/scala/spark/ShuffleSuite.scala25
-rw-r--r--core/src/test/scala/spark/SortingSuite.scala10
-rw-r--r--core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala (renamed from core/src/test/scala/spark/ParallelCollectionSplitSuite.scala)40
-rw-r--r--core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala22
-rw-r--r--core/src/test/scala/spark/scheduler/TaskContextSuite.scala10
77 files changed, 744 insertions, 704 deletions
diff --git a/core/src/main/scala/spark/CacheManager.scala b/core/src/main/scala/spark/CacheManager.scala
index 711435c333..c7b379a3fb 100644
--- a/core/src/main/scala/spark/CacheManager.scala
+++ b/core/src/main/scala/spark/CacheManager.scala
@@ -11,13 +11,13 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging {
private val loading = new HashSet[String]
/** Gets or computes an RDD split. Used by RDD.iterator() when an RDD is cached. */
- def getOrCompute[T](rdd: RDD[T], split: Split, context: TaskContext, storageLevel: StorageLevel)
+ def getOrCompute[T](rdd: RDD[T], split: Partition, context: TaskContext, storageLevel: StorageLevel)
: Iterator[T] = {
val key = "rdd_%d_%d".format(rdd.id, split.index)
logInfo("Cache key is " + key)
blockManager.get(key) match {
case Some(cachedValues) =>
- // Split is in cache, so just return its values
+ // Partition is in cache, so just return its values
logInfo("Found partition in cache!")
return cachedValues.asInstanceOf[Iterator[T]]
diff --git a/core/src/main/scala/spark/DoubleRDDFunctions.scala b/core/src/main/scala/spark/DoubleRDDFunctions.scala
index b2a0e2b631..178d31a73b 100644
--- a/core/src/main/scala/spark/DoubleRDDFunctions.scala
+++ b/core/src/main/scala/spark/DoubleRDDFunctions.scala
@@ -42,14 +42,14 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
/** (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)
+ val evaluator = new MeanEvaluator(self.partitions.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)
+ val evaluator = new SumEvaluator(self.partitions.size, confidence)
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
}
}
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala
index cc3cca2571..4319cbd892 100644
--- a/core/src/main/scala/spark/PairRDDFunctions.scala
+++ b/core/src/main/scala/spark/PairRDDFunctions.scala
@@ -62,7 +62,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
}
val aggregator =
new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners)
- if (mapSideCombine) {
+ if (self.partitioner == Some(partitioner)) {
+ self.mapPartitions(aggregator.combineValuesByKey(_), true)
+ } else if (mapSideCombine) {
val mapSideCombined = self.mapPartitions(aggregator.combineValuesByKey(_), true)
val partitioned = new ShuffledRDD[K, C](mapSideCombined, partitioner)
partitioned.mapPartitions(aggregator.combineCombinersByKey(_), true)
@@ -81,8 +83,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
def combineByKey[C](createCombiner: V => C,
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C,
- numSplits: Int): RDD[(K, C)] = {
- combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numSplits))
+ numPartitions: Int): RDD[(K, C)] = {
+ combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions))
}
/**
@@ -143,10 +145,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
/**
* 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.
+ * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.
*/
- def reduceByKey(func: (V, V) => V, numSplits: Int): RDD[(K, V)] = {
- reduceByKey(new HashPartitioner(numSplits), func)
+ def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = {
+ reduceByKey(new HashPartitioner(numPartitions), func)
}
/**
@@ -164,10 +166,10 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
/**
* Group the values for each key in the RDD into a single sequence. Hash-partitions the
- * resulting RDD with into `numSplits` partitions.
+ * resulting RDD with into `numPartitions` partitions.
*/
- def groupByKey(numSplits: Int): RDD[(K, Seq[V])] = {
- groupByKey(new HashPartitioner(numSplits))
+ def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = {
+ groupByKey(new HashPartitioner(numPartitions))
}
/**
@@ -285,8 +287,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
* 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))
+ def join[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, W))] = {
+ join(other, new HashPartitioner(numPartitions))
}
/**
@@ -303,10 +305,10 @@ 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. Hash-partitions the output
- * into `numSplits` partitions.
+ * into `numPartitions` partitions.
*/
- def leftOuterJoin[W](other: RDD[(K, W)], numSplits: Int): RDD[(K, (V, Option[W]))] = {
- leftOuterJoin(other, new HashPartitioner(numSplits))
+ def leftOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (V, Option[W]))] = {
+ leftOuterJoin(other, new HashPartitioner(numPartitions))
}
/**
@@ -325,8 +327,8 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
* 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))
+ def rightOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], W))] = {
+ rightOuterJoin(other, new HashPartitioner(numPartitions))
}
/**
@@ -361,7 +363,7 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](
- Seq(self.asInstanceOf[RDD[(_, _)]], other.asInstanceOf[RDD[(_, _)]]),
+ Seq(self.asInstanceOf[RDD[(K, _)]], other.asInstanceOf[RDD[(K, _)]]),
partitioner)
val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
prfs.mapValues {
@@ -380,9 +382,9 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest](
throw new SparkException("Default partitioner cannot partition array keys.")
}
val cg = new CoGroupedRDD[K](
- Seq(self.asInstanceOf[RDD[(_, _)]],
- other1.asInstanceOf[RDD[(_, _)]],
- other2.asInstanceOf[RDD[(_, _)]]),
+ Seq(self.asInstanceOf[RDD[(K, _)]],
+ other1.asInstanceOf[RDD[(K, _)]],
+ other2.asInstanceOf[RDD[(K, _)]]),
partitioner)
val prfs = new PairRDDFunctions[K, Seq[Seq[_]]](cg)(classManifest[K], Manifests.seqSeqManifest)
prfs.mapValues {
@@ -412,17 +414,17 @@ 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)], numSplits: Int): RDD[(K, (Seq[V], Seq[W]))] = {
- cogroup(other, new HashPartitioner(numSplits))
+ def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = {
+ cogroup(other, new HashPartitioner(numPartitions))
}
/**
* 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)
+ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int)
: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = {
- cogroup(other1, other2, new HashPartitioner(numSplits))
+ cogroup(other1, other2, new HashPartitioner(numPartitions))
}
/** Alias for cogroup. */
@@ -634,9 +636,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
* (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)] = {
+ def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[(K,V)] = {
val shuffled =
- new ShuffledRDD[K, V](self, new RangePartitioner(numSplits, self, ascending))
+ new ShuffledRDD[K, V](self, new RangePartitioner(numPartitions, self, ascending))
shuffled.mapPartitions(iter => {
val buf = iter.toArray
if (ascending) {
@@ -650,9 +652,9 @@ class OrderedRDDFunctions[K <% Ordered[K]: ClassManifest, V: ClassManifest](
private[spark]
class MappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => U) extends RDD[(K, U)](prev) {
- override def getSplits = firstParent[(K, V)].splits
+ override def getPartitions = firstParent[(K, V)].partitions
override val partitioner = firstParent[(K, V)].partitioner
- override def compute(split: Split, context: TaskContext) =
+ override def compute(split: Partition, context: TaskContext) =
firstParent[(K, V)].iterator(split, context).map{ case (k, v) => (k, f(v)) }
}
@@ -660,9 +662,9 @@ private[spark]
class FlatMappedValuesRDD[K, V, U](prev: RDD[(K, V)], f: V => TraversableOnce[U])
extends RDD[(K, U)](prev) {
- override def getSplits = firstParent[(K, V)].splits
+ override def getPartitions = firstParent[(K, V)].partitions
override val partitioner = firstParent[(K, V)].partitioner
- override def compute(split: Split, context: TaskContext) = {
+ override def compute(split: Partition, context: TaskContext) = {
firstParent[(K, V)].iterator(split, context).flatMap { case (k, v) => f(v).map(x => (k, x)) }
}
}
diff --git a/core/src/main/scala/spark/Split.scala b/core/src/main/scala/spark/Partition.scala
index 90d4b47c55..e384308ef6 100644
--- a/core/src/main/scala/spark/Split.scala
+++ b/core/src/main/scala/spark/Partition.scala
@@ -3,7 +3,7 @@ package spark
/**
* A partition of an RDD.
*/
-trait Split extends Serializable {
+trait Partition extends Serializable {
/**
* Get the split's index within its parent RDD
*/
diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala
index f6e927a989..da82dfd10f 100644
--- a/core/src/main/scala/spark/RDD.scala
+++ b/core/src/main/scala/spark/RDD.scala
@@ -27,7 +27,7 @@ import spark.rdd.FlatMappedRDD
import spark.rdd.GlommedRDD
import spark.rdd.MappedRDD
import spark.rdd.MapPartitionsRDD
-import spark.rdd.MapPartitionsWithSplitRDD
+import spark.rdd.MapPartitionsWithIndexRDD
import spark.rdd.PipedRDD
import spark.rdd.SampledRDD
import spark.rdd.UnionRDD
@@ -49,7 +49,7 @@ import SparkContext._
*
* Internally, each RDD is characterized by five main properties:
*
- * - A list of splits (partitions)
+ * - A list of 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)
@@ -76,13 +76,13 @@ abstract class RDD[T: ClassManifest](
// =======================================================================
/** Implemented by subclasses to compute a given partition. */
- def compute(split: Split, context: TaskContext): Iterator[T]
+ def compute(split: Partition, context: TaskContext): Iterator[T]
/**
* Implemented by subclasses to return the set of partitions in this RDD. This method will only
* be called once, so it is safe to implement a time-consuming computation in it.
*/
- protected def getSplits: Array[Split]
+ protected def getPartitions: Array[Partition]
/**
* Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only
@@ -91,7 +91,7 @@ abstract class RDD[T: ClassManifest](
protected def getDependencies: Seq[Dependency[_]] = deps
/** Optionally overridden by subclasses to specify placement preferences. */
- protected def getPreferredLocations(split: Split): Seq[String] = Nil
+ protected def getPreferredLocations(split: Partition): Seq[String] = Nil
/** Optionally overridden by subclasses to specify how they are partitioned. */
val partitioner: Option[Partitioner] = None
@@ -137,10 +137,10 @@ abstract class RDD[T: ClassManifest](
/** Get the RDD's current storage level, or StorageLevel.NONE if none is set. */
def getStorageLevel = storageLevel
- // Our dependencies and splits will be gotten by calling subclass's methods below, and will
+ // Our dependencies and partitions will be gotten by calling subclass's methods below, and will
// be overwritten when we're checkpointed
private var dependencies_ : Seq[Dependency[_]] = null
- @transient private var splits_ : Array[Split] = null
+ @transient private var partitions_ : Array[Partition] = null
/** An Option holding our checkpoint RDD, if we are checkpointed */
private def checkpointRDD: Option[RDD[T]] = checkpointData.flatMap(_.checkpointRDD)
@@ -159,15 +159,15 @@ abstract class RDD[T: ClassManifest](
}
/**
- * Get the array of splits of this RDD, taking into account whether the
+ * Get the array of partitions of this RDD, taking into account whether the
* RDD is checkpointed or not.
*/
- final def splits: Array[Split] = {
- checkpointRDD.map(_.splits).getOrElse {
- if (splits_ == null) {
- splits_ = getSplits
+ final def partitions: Array[Partition] = {
+ checkpointRDD.map(_.partitions).getOrElse {
+ if (partitions_ == null) {
+ partitions_ = getPartitions
}
- splits_
+ partitions_
}
}
@@ -175,7 +175,7 @@ abstract class RDD[T: ClassManifest](
* Get the preferred location of a split, taking into account whether the
* RDD is checkpointed or not.
*/
- final def preferredLocations(split: Split): Seq[String] = {
+ final def preferredLocations(split: Partition): Seq[String] = {
checkpointRDD.map(_.getPreferredLocations(split)).getOrElse {
getPreferredLocations(split)
}
@@ -186,7 +186,7 @@ abstract class RDD[T: ClassManifest](
* This should ''not'' be called by users directly, but is available for implementors of custom
* subclasses of RDD.
*/
- final def iterator(split: Split, context: TaskContext): Iterator[T] = {
+ final def iterator(split: Partition, context: TaskContext): Iterator[T] = {
if (storageLevel != StorageLevel.NONE) {
SparkEnv.get.cacheManager.getOrCompute(this, split, context, storageLevel)
} else {
@@ -197,7 +197,7 @@ abstract class RDD[T: ClassManifest](
/**
* Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing.
*/
- private[spark] def computeOrReadCheckpoint(split: Split, context: TaskContext): Iterator[T] = {
+ private[spark] def computeOrReadCheckpoint(split: Partition, context: TaskContext): Iterator[T] = {
if (isCheckpointed) {
firstParent[T].iterator(split, context)
} else {
@@ -227,15 +227,15 @@ abstract class RDD[T: ClassManifest](
/**
* Return a new RDD containing the distinct elements in this RDD.
*/
- def distinct(numSplits: Int): RDD[T] =
- map(x => (x, null)).reduceByKey((x, y) => x, numSplits).map(_._1)
+ def distinct(numPartitions: Int): RDD[T] =
+ map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1)
- def distinct(): RDD[T] = distinct(splits.size)
+ def distinct(): RDD[T] = distinct(partitions.size)
/**
- * Return a new RDD that is reduced into `numSplits` partitions.
+ * Return a new RDD that is reduced into `numPartitions` partitions.
*/
- def coalesce(numSplits: Int): RDD[T] = new CoalescedRDD(this, numSplits)
+ def coalesce(numPartitions: Int): RDD[T] = new CoalescedRDD(this, numPartitions)
/**
* Return a sampled subset of this RDD.
@@ -303,9 +303,9 @@ abstract class RDD[T: 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: ClassManifest](f: T => K, numSplits: Int): RDD[(K, Seq[T])] = {
+ def groupBy[K: ClassManifest](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = {
val cleanF = sc.clean(f)
- this.map(t => (cleanF(t), t)).groupByKey(numSplits)
+ this.map(t => (cleanF(t), t)).groupByKey(numPartitions)
}
/**
@@ -336,14 +336,24 @@ abstract class RDD[T: ClassManifest](
preservesPartitioning: Boolean = false): RDD[U] =
new MapPartitionsRDD(this, sc.clean(f), preservesPartitioning)
- /**
+ /**
+ * Return a new RDD by applying a function to each partition of this RDD, while tracking the index
+ * of the original partition.
+ */
+ def mapPartitionsWithIndex[U: ClassManifest](
+ f: (Int, Iterator[T]) => Iterator[U],
+ preservesPartitioning: Boolean = false): RDD[U] =
+ new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
+
+ /**
* Return a new RDD by applying a function to each partition of this RDD, while tracking the index
* of the original partition.
*/
+ @deprecated("use mapPartitionsWithIndex")
def mapPartitionsWithSplit[U: ClassManifest](
f: (Int, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean = false): RDD[U] =
- new MapPartitionsWithSplitRDD(this, sc.clean(f), preservesPartitioning)
+ new MapPartitionsWithIndexRDD(this, sc.clean(f), preservesPartitioning)
/**
* Zips this RDD with another one, returning key-value pairs with the first element in each RDD,
@@ -471,7 +481,7 @@ abstract class RDD[T: ClassManifest](
}
result
}
- val evaluator = new CountEvaluator(splits.size, confidence)
+ val evaluator = new CountEvaluator(partitions.size, confidence)
sc.runApproximateJob(this, countElements, evaluator, timeout)
}
@@ -522,7 +532,7 @@ abstract class RDD[T: ClassManifest](
}
map
}
- val evaluator = new GroupedCountEvaluator[T](splits.size, confidence)
+ val evaluator = new GroupedCountEvaluator[T](partitions.size, confidence)
sc.runApproximateJob(this, countPartition, evaluator, timeout)
}
@@ -537,7 +547,7 @@ abstract class RDD[T: ClassManifest](
}
val buf = new ArrayBuffer[T]
var p = 0
- while (buf.size < num && p < splits.size) {
+ while (buf.size < num && p < partitions.size) {
val left = num - buf.size
val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, Array(p), true)
buf ++= res(0)
@@ -657,11 +667,11 @@ abstract class RDD[T: ClassManifest](
/**
* Changes the dependencies of this RDD from its original parents to a new RDD (`newRDD`)
- * created from the checkpoint file, and forget its old dependencies and splits.
+ * created from the checkpoint file, and forget its old dependencies and partitions.
*/
private[spark] def markCheckpointed(checkpointRDD: RDD[_]) {
clearDependencies()
- splits_ = null
+ partitions_ = null
deps = null // Forget the constructor argument for dependencies too
}
@@ -676,15 +686,15 @@ abstract class RDD[T: ClassManifest](
}
/** A description of this RDD and its recursive dependencies for debugging. */
- def toDebugString(): String = {
+ def toDebugString: String = {
def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = {
- Seq(prefix + rdd + " (" + rdd.splits.size + " splits)") ++
+ Seq(prefix + rdd + " (" + rdd.partitions.size + " partitions)") ++
rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " "))
}
debugString(this).mkString("\n")
}
- override def toString(): String = "%s%s[%d] at %s".format(
+ override def toString: String = "%s%s[%d] at %s".format(
Option(name).map(_ + " ").getOrElse(""),
getClass.getSimpleName,
id,
diff --git a/core/src/main/scala/spark/RDDCheckpointData.scala b/core/src/main/scala/spark/RDDCheckpointData.scala
index a4a4ebaf53..d00092e984 100644
--- a/core/src/main/scala/spark/RDDCheckpointData.scala
+++ b/core/src/main/scala/spark/RDDCheckpointData.scala
@@ -16,7 +16,7 @@ private[spark] object CheckpointState extends Enumeration {
/**
* This class contains all the information related to RDD checkpointing. Each instance of this class
* is associated with a RDD. It manages process of checkpointing of the associated RDD, as well as,
- * manages the post-checkpoint state by providing the updated splits, iterator and preferred locations
+ * manages the post-checkpoint state by providing the updated partitions, iterator and preferred locations
* of the checkpointed RDD.
*/
private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
@@ -67,11 +67,11 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
rdd.context.runJob(rdd, CheckpointRDD.writeToFile(path) _)
val newRDD = new CheckpointRDD[T](rdd.context, path)
- // Change the dependencies and splits of the RDD
+ // Change the dependencies and partitions of the RDD
RDDCheckpointData.synchronized {
cpFile = Some(path)
cpRDD = Some(newRDD)
- rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and splits
+ rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions
cpState = Checkpointed
RDDCheckpointData.clearTaskCaches()
logInfo("Done checkpointing RDD " + rdd.id + ", new parent is RDD " + newRDD.id)
@@ -79,15 +79,15 @@ private[spark] class RDDCheckpointData[T: ClassManifest](rdd: RDD[T])
}
// Get preferred location of a split after checkpointing
- def getPreferredLocations(split: Split): Seq[String] = {
+ def getPreferredLocations(split: Partition): Seq[String] = {
RDDCheckpointData.synchronized {
cpRDD.get.preferredLocations(split)
}
}
- def getSplits: Array[Split] = {
+ def getPartitions: Array[Partition] = {
RDDCheckpointData.synchronized {
- cpRDD.get.splits
+ cpRDD.get.partitions
}
}
diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala
index 3b5cc58af7..f40bb7935f 100644
--- a/core/src/main/scala/spark/SparkContext.scala
+++ b/core/src/main/scala/spark/SparkContext.scala
@@ -39,7 +39,7 @@ import spark.broadcast._
import spark.deploy.LocalSparkCluster
import spark.partial.ApproximateEvaluator
import spark.partial.PartialResult
-import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD}
+import rdd.{CheckpointRDD, HadoopRDD, NewHadoopRDD, UnionRDD, ParallelCollectionRDD}
import scheduler.{ResultTask, ShuffleMapTask, DAGScheduler, TaskScheduler}
import spark.scheduler.local.LocalScheduler
import spark.scheduler.cluster.{SparkDeploySchedulerBackend, SchedulerBackend, ClusterScheduler}
@@ -53,7 +53,7 @@ import storage.{StorageStatus, StorageUtils, RDDInfo}
* 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 appName A name for your application, to display on the cluster web UI.
* @param sparkHome Location where Spark is installed on cluster nodes.
* @param jars Collection of JARs to send to the cluster. These can be paths on the local file
* system or HDFS, HTTP, HTTPS, or FTP URLs.
@@ -61,7 +61,7 @@ import storage.{StorageStatus, StorageUtils, RDDInfo}
*/
class SparkContext(
val master: String,
- val jobName: String,
+ val appName: String,
val sparkHome: String = null,
val jars: Seq[String] = Nil,
environment: Map[String, String] = Map())
@@ -143,7 +143,7 @@ class SparkContext(
case SPARK_REGEX(sparkUrl) =>
val scheduler = new ClusterScheduler(this)
- val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName)
+ val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
scheduler.initialize(backend)
scheduler
@@ -162,7 +162,7 @@ class SparkContext(
val localCluster = new LocalSparkCluster(
numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt)
val sparkUrl = localCluster.start()
- val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, jobName)
+ val backend = new SparkDeploySchedulerBackend(scheduler, this, sparkUrl, appName)
scheduler.initialize(backend)
backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => {
localCluster.stop()
@@ -178,9 +178,9 @@ class SparkContext(
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, masterWithoutProtocol, jobName)
+ new CoarseMesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName)
} else {
- new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, jobName)
+ new MesosSchedulerBackend(scheduler, this, masterWithoutProtocol, appName)
}
scheduler.initialize(backend)
scheduler
@@ -216,7 +216,7 @@ class SparkContext(
/** 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, Map[Int, Seq[String]]())
+ new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]())
}
/** Distribute a local Scala collection to form an RDD. */
@@ -229,7 +229,7 @@ class SparkContext(
* Create a new partition for each collection item. */
def makeRDD[T: ClassManifest](seq: Seq[(T, Seq[String])]): RDD[T] = {
val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
- new ParallelCollection[T](this, seq.map(_._1), seq.size, indexToPrefs)
+ new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs)
}
/**
@@ -614,14 +614,14 @@ class SparkContext(
* Run a job on all partitions in an RDD and return the results in an array.
*/
def runJob[T, U: ClassManifest](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = {
- runJob(rdd, func, 0 until rdd.splits.size, false)
+ runJob(rdd, func, 0 until rdd.partitions.size, false)
}
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
def runJob[T, U: ClassManifest](rdd: RDD[T], func: Iterator[T] => U): Array[U] = {
- runJob(rdd, func, 0 until rdd.splits.size, false)
+ runJob(rdd, func, 0 until rdd.partitions.size, false)
}
/**
@@ -632,7 +632,7 @@ class SparkContext(
processPartition: (TaskContext, Iterator[T]) => U,
resultHandler: (Int, U) => Unit)
{
- runJob[T, U](rdd, processPartition, 0 until rdd.splits.size, false, resultHandler)
+ runJob[T, U](rdd, processPartition, 0 until rdd.partitions.size, false, resultHandler)
}
/**
@@ -644,7 +644,7 @@ class SparkContext(
resultHandler: (Int, U) => Unit)
{
val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter)
- runJob[T, U](rdd, processFunc, 0 until rdd.splits.size, false, resultHandler)
+ runJob[T, U](rdd, processFunc, 0 until rdd.partitions.size, false, resultHandler)
}
/**
@@ -696,7 +696,7 @@ class SparkContext(
/** 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 partitions for Hadoop RDDs when not given by user */
def defaultMinSplits: Int = math.min(defaultParallelism, 2)
private var nextShuffleId = new AtomicInteger(0)
diff --git a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
index 2810631b41..da3cb2cd31 100644
--- a/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaDoubleRDD.scala
@@ -44,7 +44,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
/**
* Return a new RDD containing the distinct elements in this RDD.
*/
- def distinct(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numSplits))
+ def distinct(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.distinct(numPartitions))
/**
* Return a new RDD containing only the elements that satisfy a predicate.
@@ -53,9 +53,9 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[Double, Jav
fromRDD(srdd.filter(x => f(x).booleanValue()))
/**
- * Return a new RDD that is reduced into `numSplits` partitions.
+ * Return a new RDD that is reduced into `numPartitions` partitions.
*/
- def coalesce(numSplits: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numSplits))
+ def coalesce(numPartitions: Int): JavaDoubleRDD = fromRDD(srdd.coalesce(numPartitions))
/**
* Return a sampled subset of this RDD.
diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
index 55dc755358..df3af3817d 100644
--- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala
@@ -54,7 +54,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/**
* 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))
+ def distinct(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.distinct(numPartitions))
/**
* Return a new RDD containing only the elements that satisfy a predicate.
@@ -63,9 +63,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue()))
/**
- * Return a new RDD that is reduced into `numSplits` partitions.
+ * Return a new RDD that is reduced into `numPartitions` partitions.
*/
- def coalesce(numSplits: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.coalesce(numSplits))
+ def coalesce(numPartitions: Int): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.coalesce(numPartitions))
/**
* Return a sampled subset of this RDD.
@@ -122,8 +122,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
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))
+ numPartitions: Int): JavaPairRDD[K, C] =
+ combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions))
/**
* Merge the values for each key using an associative reduce function. This will also perform
@@ -162,10 +162,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/**
* 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.
+ * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions.
*/
- def reduceByKey(func: JFunction2[V, V, V], numSplits: Int): JavaPairRDD[K, V] =
- fromRDD(rdd.reduceByKey(func, numSplits))
+ def reduceByKey(func: JFunction2[V, V, V], numPartitions: Int): JavaPairRDD[K, V] =
+ fromRDD(rdd.reduceByKey(func, numPartitions))
/**
* Group the values for each key in the RDD into a single sequence. Allows controlling the
@@ -176,10 +176,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
/**
* Group the values for each key in the RDD into a single sequence. Hash-partitions the
- * resulting RDD with into `numSplits` partitions.
+ * resulting RDD with into `numPartitions` partitions.
*/
- def groupByKey(numSplits: Int): JavaPairRDD[K, JList[V]] =
- fromRDD(groupByResultToJava(rdd.groupByKey(numSplits)))
+ def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] =
+ fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions)))
/**
* Return a copy of the RDD partitioned using the specified partitioner. If `mapSideCombine`
@@ -261,8 +261,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* 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))
+ def join[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, W)] =
+ fromRDD(rdd.join(other, numPartitions))
/**
* Perform a left outer join of `this` and `other`. For each element (k, v) in `this`, the
@@ -277,10 +277,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* 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.
+ * into `numPartitions` partitions.
*/
- def leftOuterJoin[W](other: JavaPairRDD[K, W], numSplits: Int): JavaPairRDD[K, (V, Option[W])] =
- fromRDD(rdd.leftOuterJoin(other, numSplits))
+ def leftOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (V, Option[W])] =
+ fromRDD(rdd.leftOuterJoin(other, numPartitions))
/**
* Perform a right outer join of `this` and `other`. For each element (k, w) in `other`, the
@@ -297,8 +297,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* 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))
+ def rightOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (Option[V], W)] =
+ fromRDD(rdd.rightOuterJoin(other, numPartitions))
/**
* Return the key-value pairs in this RDD to the master as a Map.
@@ -362,16 +362,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif
* 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)))
+ def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int): JavaPairRDD[K, (JList[V], JList[W])]
+ = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions)))
/**
* 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)
+ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int)
: JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] =
- fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numSplits)))
+ fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions)))
/** Alias for cogroup. */
def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] =
diff --git a/core/src/main/scala/spark/api/java/JavaRDD.scala b/core/src/main/scala/spark/api/java/JavaRDD.scala
index 23e7ae2726..3ccd6f055e 100644
--- a/core/src/main/scala/spark/api/java/JavaRDD.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDD.scala
@@ -30,7 +30,7 @@ JavaRDDLike[T, JavaRDD[T]] {
/**
* Return a new RDD containing the distinct elements in this RDD.
*/
- def distinct(numSplits: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numSplits))
+ def distinct(numPartitions: Int): JavaRDD[T] = wrapRDD(rdd.distinct(numPartitions))
/**
* Return a new RDD containing only the elements that satisfy a predicate.
@@ -39,9 +39,9 @@ JavaRDDLike[T, JavaRDD[T]] {
wrapRDD(rdd.filter((x => f(x).booleanValue())))
/**
- * Return a new RDD that is reduced into `numSplits` partitions.
+ * Return a new RDD that is reduced into `numPartitions` partitions.
*/
- def coalesce(numSplits: Int): JavaRDD[T] = rdd.coalesce(numSplits)
+ def coalesce(numPartitions: Int): JavaRDD[T] = rdd.coalesce(numPartitions)
/**
* Return a sampled subset of this RDD.
diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
index d34d56d169..90b45cf875 100644
--- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala
@@ -4,7 +4,7 @@ import java.util.{List => JList}
import scala.Tuple2
import scala.collection.JavaConversions._
-import spark.{SparkContext, Split, RDD, TaskContext}
+import spark.{SparkContext, Partition, RDD, TaskContext}
import spark.api.java.JavaPairRDD._
import spark.api.java.function.{Function2 => JFunction2, Function => JFunction, _}
import spark.partial.{PartialResult, BoundedDouble}
@@ -20,7 +20,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround
def rdd: RDD[T]
/** Set of partitions in this RDD. */
- def splits: JList[Split] = new java.util.ArrayList(rdd.splits.toSeq)
+ def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq)
/** The [[spark.SparkContext]] that this RDD was created on. */
def context: SparkContext = rdd.context
@@ -36,7 +36,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround
* This should ''not'' be called by users directly, but is available for implementors of custom
* subclasses of RDD.
*/
- def iterator(split: Split, taskContext: TaskContext): java.util.Iterator[T] =
+ def iterator(split: Partition, taskContext: TaskContext): java.util.Iterator[T] =
asJavaIterator(rdd.iterator(split, taskContext))
// Transformations (return a new RDD)
@@ -146,12 +146,12 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround
* 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]] = {
+ def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = {
implicit val kcm: ClassManifest[K] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]]
implicit val vcm: ClassManifest[JList[T]] =
implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[JList[T]]]
- JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numSplits)(f.returnType)))(kcm, vcm)
+ JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType)))(kcm, vcm)
}
/**
@@ -333,6 +333,6 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends PairFlatMapWorkaround
/** A description of this RDD and its recursive dependencies for debugging. */
def toDebugString(): String = {
- rdd.toDebugString()
+ rdd.toDebugString
}
}
diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
index 50b8970cd8..f75fc27c7b 100644
--- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala
@@ -23,41 +23,41 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
/**
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param jobName A name for your job, to display on the cluster web UI
+ * @param appName A name for your application, to display on the cluster web UI
*/
- def this(master: String, jobName: String) = this(new SparkContext(master, jobName))
+ def this(master: String, appName: String) = this(new SparkContext(master, appName))
/**
* @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]).
- * @param jobName A name for your job, to display on the cluster web UI
+ * @param appName A name for your application, 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, appName: String, sparkHome: String, jarFile: String) =
+ this(new SparkContext(master, appName, 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 appName A name for your application, 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, appName: String, sparkHome: String, jars: Array[String]) =
+ this(new SparkContext(master, appName, 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 appName A name for your application, 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],
+ def this(master: String, appName: String, sparkHome: String, jars: Array[String],
environment: JMap[String, String]) =
- this(new SparkContext(master, jobName, sparkHome, jars.toSeq, environment))
+ this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment))
private[spark] val env = sc.env
diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala
index 519e310323..d618c098c2 100644
--- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala
+++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala
@@ -9,7 +9,7 @@ import java.util.Arrays
*
* Stores the unique id() of the Python-side partitioning function so that it is incorporated into
* equality comparisons. Correctness requires that the id is a unique identifier for the
- * lifetime of the job (i.e. that it is not re-used as the id of a different partitioning
+ * lifetime of the program (i.e. that it is not re-used as the id of a different partitioning
* function). This can be ensured by using the Python id() function and maintaining a reference
* to the Python partitioning function so that its id() is not reused.
*/
diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala
index ab8351e55e..8c73477384 100644
--- a/core/src/main/scala/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/spark/api/python/PythonRDD.scala
@@ -32,11 +32,11 @@ private[spark] class PythonRDD[T: ClassManifest](
this(parent, PipedRDD.tokenize(command), envVars, preservePartitoning, pythonExec,
broadcastVars, accumulator)
- override def getSplits = parent.splits
+ override def getPartitions = parent.partitions
override val partitioner = if (preservePartitoning) parent.partitioner else None
- override def compute(split: Split, context: TaskContext): Iterator[Array[Byte]] = {
+ override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME")
val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/python/pyspark/worker.py"))
@@ -65,7 +65,7 @@ private[spark] class PythonRDD[T: ClassManifest](
SparkEnv.set(env)
val out = new PrintWriter(proc.getOutputStream)
val dOut = new DataOutputStream(proc.getOutputStream)
- // Split index
+ // Partition index
dOut.writeInt(split.index)
// sparkFilesDir
PythonRDD.writeAsPickle(SparkFiles.getRootDirectory, dOut)
@@ -155,8 +155,8 @@ private class PythonException(msg: String) extends Exception(msg)
*/
private class PairwiseRDD(prev: RDD[Array[Byte]]) extends
RDD[(Array[Byte], Array[Byte])](prev) {
- override def getSplits = prev.splits
- override def compute(split: Split, context: TaskContext) =
+ override def getPartitions = prev.partitions
+ override def compute(split: Partition, context: TaskContext) =
prev.iterator(split, context).grouped(2).map {
case Seq(a, b) => (a, b)
case x => throw new Exception("PairwiseRDD: unexpected value: " + x)
diff --git a/core/src/main/scala/spark/deploy/JobDescription.scala b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
index 7160fc05fc..6659e53b25 100644
--- a/core/src/main/scala/spark/deploy/JobDescription.scala
+++ b/core/src/main/scala/spark/deploy/ApplicationDescription.scala
@@ -1,6 +1,6 @@
package spark.deploy
-private[spark] class JobDescription(
+private[spark] class ApplicationDescription(
val name: String,
val cores: Int,
val memoryPerSlave: Int,
@@ -10,5 +10,5 @@ private[spark] class JobDescription(
val user = System.getProperty("user.name", "<unknown>")
- override def toString: String = "JobDescription(" + name + ")"
+ override def toString: String = "ApplicationDescription(" + name + ")"
}
diff --git a/core/src/main/scala/spark/deploy/DeployMessage.scala b/core/src/main/scala/spark/deploy/DeployMessage.scala
index 1d88d4bc84..3cbf4fdd98 100644
--- a/core/src/main/scala/spark/deploy/DeployMessage.scala
+++ b/core/src/main/scala/spark/deploy/DeployMessage.scala
@@ -1,7 +1,7 @@
package spark.deploy
import spark.deploy.ExecutorState.ExecutorState
-import spark.deploy.master.{WorkerInfo, JobInfo}
+import spark.deploy.master.{WorkerInfo, ApplicationInfo}
import spark.deploy.worker.ExecutorRunner
import scala.collection.immutable.List
@@ -23,7 +23,7 @@ case class RegisterWorker(
private[spark]
case class ExecutorStateChanged(
- jobId: String,
+ appId: String,
execId: Int,
state: ExecutorState,
message: Option[String],
@@ -36,12 +36,12 @@ private[spark] case class Heartbeat(workerId: String) 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
+private[spark] case class KillExecutor(appId: String, execId: Int) extends DeployMessage
private[spark] case class LaunchExecutor(
- jobId: String,
+ appId: String,
execId: Int,
- jobDesc: JobDescription,
+ appDesc: ApplicationDescription,
cores: Int,
memory: Int,
sparkHome: String)
@@ -49,12 +49,13 @@ private[spark] case class LaunchExecutor(
// Client to Master
-private[spark] case class RegisterJob(jobDescription: JobDescription) extends DeployMessage
+private[spark] case class RegisterApplication(appDescription: ApplicationDescription)
+ extends DeployMessage
// Master to Client
private[spark]
-case class RegisteredJob(jobId: String) extends DeployMessage
+case class RegisteredApplication(appId: String) extends DeployMessage
private[spark]
case class ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int)
@@ -64,7 +65,7 @@ case class ExecutorUpdated(id: Int, state: ExecutorState, message: Option[String
exitStatus: Option[Int])
private[spark]
-case class JobKilled(message: String)
+case class appKilled(message: String)
// Internal message in Client
@@ -78,7 +79,7 @@ private[spark] case object RequestMasterState
private[spark]
case class MasterState(host: String, port: Int, workers: Array[WorkerInfo],
- activeJobs: Array[JobInfo], completedJobs: Array[JobInfo]) {
+ activeApps: Array[ApplicationInfo], completedApps: Array[ApplicationInfo]) {
def uri = "spark://" + host + ":" + port
}
diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala
index 732fa08064..38a6ebfc24 100644
--- a/core/src/main/scala/spark/deploy/JsonProtocol.scala
+++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala
@@ -1,6 +1,6 @@
package spark.deploy
-import master.{JobInfo, WorkerInfo}
+import master.{ApplicationInfo, WorkerInfo}
import worker.ExecutorRunner
import cc.spray.json._
@@ -20,8 +20,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
)
}
- implicit object JobInfoJsonFormat extends RootJsonWriter[JobInfo] {
- def write(obj: JobInfo) = JsObject(
+ implicit object AppInfoJsonFormat extends RootJsonWriter[ApplicationInfo] {
+ def write(obj: ApplicationInfo) = JsObject(
"starttime" -> JsNumber(obj.startTime),
"id" -> JsString(obj.id),
"name" -> JsString(obj.desc.name),
@@ -31,8 +31,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
"submitdate" -> JsString(obj.submitDate.toString))
}
- implicit object JobDescriptionJsonFormat extends RootJsonWriter[JobDescription] {
- def write(obj: JobDescription) = JsObject(
+ implicit object AppDescriptionJsonFormat extends RootJsonWriter[ApplicationDescription] {
+ def write(obj: ApplicationDescription) = JsObject(
"name" -> JsString(obj.name),
"cores" -> JsNumber(obj.cores),
"memoryperslave" -> JsNumber(obj.memoryPerSlave),
@@ -44,8 +44,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
def write(obj: ExecutorRunner) = JsObject(
"id" -> JsNumber(obj.execId),
"memory" -> JsNumber(obj.memory),
- "jobid" -> JsString(obj.jobId),
- "jobdesc" -> obj.jobDesc.toJson.asJsObject
+ "appid" -> JsString(obj.appId),
+ "appdesc" -> obj.appDesc.toJson.asJsObject
)
}
@@ -57,8 +57,8 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol {
"coresused" -> JsNumber(obj.workers.map(_.coresUsed).sum),
"memory" -> JsNumber(obj.workers.map(_.memory).sum),
"memoryused" -> JsNumber(obj.workers.map(_.memoryUsed).sum),
- "activejobs" -> JsArray(obj.activeJobs.toList.map(_.toJson)),
- "completedjobs" -> JsArray(obj.completedJobs.toList.map(_.toJson))
+ "activeapps" -> JsArray(obj.activeApps.toList.map(_.toJson)),
+ "completedapps" -> JsArray(obj.completedApps.toList.map(_.toJson))
)
}
diff --git a/core/src/main/scala/spark/deploy/client/Client.scala b/core/src/main/scala/spark/deploy/client/Client.scala
index e01181d1b2..1a95524cf9 100644
--- a/core/src/main/scala/spark/deploy/client/Client.scala
+++ b/core/src/main/scala/spark/deploy/client/Client.scala
@@ -8,25 +8,25 @@ import akka.pattern.AskTimeoutException
import spark.{SparkException, Logging}
import akka.remote.RemoteClientLifeCycleEvent
import akka.remote.RemoteClientShutdown
-import spark.deploy.RegisterJob
+import spark.deploy.RegisterApplication
import spark.deploy.master.Master
import akka.remote.RemoteClientDisconnected
import akka.actor.Terminated
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.
+ * The main class used to talk to a Spark deploy cluster. Takes a master URL, an app description,
+ * and a listener for cluster events, and calls back the listener when various events occur.
*/
private[spark] class Client(
actorSystem: ActorSystem,
masterUrl: String,
- jobDescription: JobDescription,
+ appDescription: ApplicationDescription,
listener: ClientListener)
extends Logging {
var actor: ActorRef = null
- var jobId: String = null
+ var appId: String = null
class ClientActor extends Actor with Logging {
var master: ActorRef = null
@@ -38,7 +38,7 @@ private[spark] class Client(
try {
master = context.actorFor(Master.toAkkaUrl(masterUrl))
masterAddress = master.path.address
- master ! RegisterJob(jobDescription)
+ master ! RegisterApplication(appDescription)
context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent])
context.watch(master) // Doesn't work with remote actors, but useful for testing
} catch {
@@ -50,17 +50,17 @@ private[spark] class Client(
}
override def receive = {
- case RegisteredJob(jobId_) =>
- jobId = jobId_
- listener.connected(jobId)
+ case RegisteredApplication(appId_) =>
+ appId = appId_
+ listener.connected(appId)
case ExecutorAdded(id: Int, workerId: String, host: String, cores: Int, memory: Int) =>
- val fullId = jobId + "/" + id
+ val fullId = appId + "/" + id
logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, host, cores))
listener.executorAdded(fullId, workerId, host, cores, memory)
case ExecutorUpdated(id, state, message, exitStatus) =>
- val fullId = jobId + "/" + id
+ val fullId = appId + "/" + id
val messageText = message.map(s => " (" + s + ")").getOrElse("")
logInfo("Executor updated: %s is now %s%s".format(fullId, state, messageText))
if (ExecutorState.isFinished(state)) {
diff --git a/core/src/main/scala/spark/deploy/client/ClientListener.scala b/core/src/main/scala/spark/deploy/client/ClientListener.scala
index 7035f4b394..b7008321df 100644
--- a/core/src/main/scala/spark/deploy/client/ClientListener.scala
+++ b/core/src/main/scala/spark/deploy/client/ClientListener.scala
@@ -8,7 +8,7 @@ package spark.deploy.client
* Users of this API should *not* block inside the callback methods.
*/
private[spark] trait ClientListener {
- def connected(jobId: String): Unit
+ def connected(appId: 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 8764c400e2..dc004b59ca 100644
--- a/core/src/main/scala/spark/deploy/client/TestClient.scala
+++ b/core/src/main/scala/spark/deploy/client/TestClient.scala
@@ -2,13 +2,13 @@ package spark.deploy.client
import spark.util.AkkaUtils
import spark.{Logging, Utils}
-import spark.deploy.{Command, JobDescription}
+import spark.deploy.{Command, ApplicationDescription}
private[spark] object TestClient {
class TestListener extends ClientListener with Logging {
def connected(id: String) {
- logInfo("Connected to master, got job ID " + id)
+ logInfo("Connected to master, got app ID " + id)
}
def disconnected() {
@@ -24,7 +24,7 @@ private[spark] object TestClient {
def main(args: Array[String]) {
val url = args(0)
val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0)
- val desc = new JobDescription(
+ val desc = new ApplicationDescription(
"TestClient", 1, 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), "dummy-spark-home")
val listener = new TestListener
val client = new Client(actorSystem, url, desc, listener)
diff --git a/core/src/main/scala/spark/deploy/master/JobInfo.scala b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
index a274b21c34..3591a94072 100644
--- a/core/src/main/scala/spark/deploy/master/JobInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ApplicationInfo.scala
@@ -1,18 +1,18 @@
package spark.deploy.master
-import spark.deploy.JobDescription
+import spark.deploy.ApplicationDescription
import java.util.Date
import akka.actor.ActorRef
import scala.collection.mutable
-private[spark] class JobInfo(
+private[spark] class ApplicationInfo(
val startTime: Long,
val id: String,
- val desc: JobDescription,
+ val desc: ApplicationDescription,
val submitDate: Date,
val driver: ActorRef)
{
- var state = JobState.WAITING
+ var state = ApplicationState.WAITING
var executors = new mutable.HashMap[Int, ExecutorInfo]
var coresGranted = 0
var endTime = -1L
@@ -48,7 +48,7 @@ private[spark] class JobInfo(
_retryCount
}
- def markFinished(endState: JobState.Value) {
+ def markFinished(endState: ApplicationState.Value) {
state = endState
endTime = System.currentTimeMillis()
}
diff --git a/core/src/main/scala/spark/deploy/master/ApplicationState.scala b/core/src/main/scala/spark/deploy/master/ApplicationState.scala
new file mode 100644
index 0000000000..15016b388d
--- /dev/null
+++ b/core/src/main/scala/spark/deploy/master/ApplicationState.scala
@@ -0,0 +1,11 @@
+package spark.deploy.master
+
+private[spark] object ApplicationState
+ extends Enumeration("WAITING", "RUNNING", "FINISHED", "FAILED") {
+
+ type ApplicationState = Value
+
+ val WAITING, RUNNING, FINISHED, FAILED = Value
+
+ val MAX_NUM_RETRY = 10
+}
diff --git a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
index 1db2c32633..48e6055fb5 100644
--- a/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/ExecutorInfo.scala
@@ -4,12 +4,12 @@ import spark.deploy.ExecutorState
private[spark] class ExecutorInfo(
val id: Int,
- val job: JobInfo,
+ val application: ApplicationInfo,
val worker: WorkerInfo,
val cores: Int,
val memory: Int) {
var state = ExecutorState.LAUNCHING
- def fullId: String = job.id + "/" + id
+ def fullId: String = application.id + "/" + id
}
diff --git a/core/src/main/scala/spark/deploy/master/JobState.scala b/core/src/main/scala/spark/deploy/master/JobState.scala
deleted file mode 100644
index 2b70cf0191..0000000000
--- a/core/src/main/scala/spark/deploy/master/JobState.scala
+++ /dev/null
@@ -1,9 +0,0 @@
-package spark.deploy.master
-
-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 a5de23261c..1cd68a2aa6 100644
--- a/core/src/main/scala/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/spark/deploy/master/Master.scala
@@ -16,22 +16,22 @@ import spark.util.AkkaUtils
private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor with Logging {
- val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For job IDs
+ val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs
val WORKER_TIMEOUT = System.getProperty("spark.worker.timeout", "60").toLong * 1000
- var nextJobNumber = 0
+ var nextAppNumber = 0
val workers = new HashSet[WorkerInfo]
val idToWorker = new HashMap[String, WorkerInfo]
val actorToWorker = new HashMap[ActorRef, WorkerInfo]
val addressToWorker = new HashMap[Address, WorkerInfo]
- val jobs = new HashSet[JobInfo]
- val idToJob = new HashMap[String, JobInfo]
- val actorToJob = new HashMap[ActorRef, JobInfo]
- val addressToJob = new HashMap[Address, JobInfo]
+ val apps = new HashSet[ApplicationInfo]
+ val idToApp = new HashMap[String, ApplicationInfo]
+ val actorToApp = new HashMap[ActorRef, ApplicationInfo]
+ val addressToApp = new HashMap[Address, ApplicationInfo]
- val waitingJobs = new ArrayBuffer[JobInfo]
- val completedJobs = new ArrayBuffer[JobInfo]
+ val waitingApps = new ArrayBuffer[ApplicationInfo]
+ val completedApps = new ArrayBuffer[ApplicationInfo]
val masterPublicAddress = {
val envVar = System.getenv("SPARK_PUBLIC_DNS")
@@ -39,9 +39,9 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
}
// As a temporary workaround before better ways of configuring memory, we allow users to set
- // a flag that will perform round-robin scheduling across the nodes (spreading out each job
- // among all the nodes) instead of trying to consolidate each job onto a small # of nodes.
- val spreadOutJobs = System.getProperty("spark.deploy.spreadOut", "false").toBoolean
+ // a flag that will perform round-robin scheduling across the nodes (spreading out each app
+ // among all the nodes) instead of trying to consolidate each app onto a small # of nodes.
+ val spreadOutApps = System.getProperty("spark.deploy.spreadOut", "false").toBoolean
override def preStart() {
logInfo("Starting Spark master at spark://" + ip + ":" + port)
@@ -76,41 +76,41 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
}
}
- case RegisterJob(description) => {
- logInfo("Registering job " + description.name)
- val job = addJob(description, sender)
- logInfo("Registered job " + description.name + " with ID " + job.id)
- waitingJobs += job
+ case RegisterApplication(description) => {
+ logInfo("Registering app " + description.name)
+ val app = addApplication(description, sender)
+ logInfo("Registered app " + description.name + " with ID " + app.id)
+ waitingApps += app
context.watch(sender) // This doesn't work with remote actors but helps for testing
- sender ! RegisteredJob(job.id)
+ sender ! RegisteredApplication(app.id)
schedule()
}
- case ExecutorStateChanged(jobId, execId, state, message, exitStatus) => {
- val execOption = idToJob.get(jobId).flatMap(job => job.executors.get(execId))
+ case ExecutorStateChanged(appId, execId, state, message, exitStatus) => {
+ val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId))
execOption match {
case Some(exec) => {
exec.state = state
- exec.job.driver ! ExecutorUpdated(execId, state, message, exitStatus)
+ exec.application.driver ! ExecutorUpdated(execId, state, message, exitStatus)
if (ExecutorState.isFinished(state)) {
- val jobInfo = idToJob(jobId)
- // Remove this executor from the worker and job
+ val appInfo = idToApp(appId)
+ // Remove this executor from the worker and app
logInfo("Removing executor " + exec.fullId + " because it is " + state)
- jobInfo.removeExecutor(exec)
+ appInfo.removeExecutor(exec)
exec.worker.removeExecutor(exec)
// Only retry certain number of times so we don't go into an infinite loop.
- if (jobInfo.incrementRetryCount < JobState.MAX_NUM_RETRY) {
+ if (appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) {
schedule()
} else {
- logError("Job %s with ID %s failed %d times, removing it".format(
- jobInfo.desc.name, jobInfo.id, jobInfo.retryCount))
- removeJob(jobInfo)
+ logError("Application %s with ID %s failed %d times, removing it".format(
+ appInfo.desc.name, appInfo.id, appInfo.retryCount))
+ removeApplication(appInfo)
}
}
}
case None =>
- logWarning("Got status update for unknown executor " + jobId + "/" + execId)
+ logWarning("Got status update for unknown executor " + appId + "/" + execId)
}
}
@@ -124,53 +124,53 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
}
case Terminated(actor) => {
- // The disconnected actor could've been either a worker or a job; remove whichever of
+ // The disconnected actor could've been either a worker or an app; remove whichever of
// those we have an entry for in the corresponding actor hashmap
actorToWorker.get(actor).foreach(removeWorker)
- actorToJob.get(actor).foreach(removeJob)
+ actorToApp.get(actor).foreach(removeApplication)
}
case RemoteClientDisconnected(transport, address) => {
- // The disconnected client could've been either a worker or a job; remove whichever it was
+ // The disconnected client could've been either a worker or an app; remove whichever it was
addressToWorker.get(address).foreach(removeWorker)
- addressToJob.get(address).foreach(removeJob)
+ addressToApp.get(address).foreach(removeApplication)
}
case RemoteClientShutdown(transport, address) => {
- // The disconnected client could've been either a worker or a job; remove whichever it was
+ // The disconnected client could've been either a worker or an app; remove whichever it was
addressToWorker.get(address).foreach(removeWorker)
- addressToJob.get(address).foreach(removeJob)
+ addressToApp.get(address).foreach(removeApplication)
}
case RequestMasterState => {
- sender ! MasterState(ip, port, workers.toArray, jobs.toArray, completedJobs.toArray)
+ sender ! MasterState(ip, port, workers.toArray, apps.toArray, completedApps.toArray)
}
}
/**
- * Can a job use the given worker? True if the worker has enough memory and we haven't already
- * launched an executor for the job on it (right now the standalone backend doesn't like having
+ * Can an app use the given worker? True if the worker has enough memory and we haven't already
+ * launched an executor for the app on it (right now the standalone backend doesn't like having
* two executors on the same worker).
*/
- def canUse(job: JobInfo, worker: WorkerInfo): Boolean = {
- worker.memoryFree >= job.desc.memoryPerSlave && !worker.hasExecutor(job)
+ def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = {
+ worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app)
}
/**
- * Schedule the currently available resources among waiting jobs. This method will be called
- * every time a new job joins or resource availability changes.
+ * Schedule the currently available resources among waiting apps. This method will be called
+ * every time a new app joins or resource availability changes.
*/
def schedule() {
- // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first job
- // in the queue, then the second job, etc.
- if (spreadOutJobs) {
- // Try to spread out each job among all the nodes, until it has all its cores
- for (job <- waitingJobs if job.coresLeft > 0) {
+ // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
+ // in the queue, then the second app, etc.
+ if (spreadOutApps) {
+ // Try to spread out each app among all the nodes, until it has all its cores
+ for (app <- waitingApps if app.coresLeft > 0) {
val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE)
- .filter(canUse(job, _)).sortBy(_.coresFree).reverse
+ .filter(canUse(app, _)).sortBy(_.coresFree).reverse
val numUsable = usableWorkers.length
val assigned = new Array[Int](numUsable) // Number of cores to give on each node
- var toAssign = math.min(job.coresLeft, usableWorkers.map(_.coresFree).sum)
+ var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum)
var pos = 0
while (toAssign > 0) {
if (usableWorkers(pos).coresFree - assigned(pos) > 0) {
@@ -182,22 +182,22 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
// Now that we've decided how many cores to give on each node, let's actually give them
for (pos <- 0 until numUsable) {
if (assigned(pos) > 0) {
- val exec = job.addExecutor(usableWorkers(pos), assigned(pos))
- launchExecutor(usableWorkers(pos), exec, job.desc.sparkHome)
- job.state = JobState.RUNNING
+ val exec = app.addExecutor(usableWorkers(pos), assigned(pos))
+ launchExecutor(usableWorkers(pos), exec, app.desc.sparkHome)
+ app.state = ApplicationState.RUNNING
}
}
}
} else {
- // Pack each job into as few nodes as possible until we've assigned all its cores
+ // Pack each app into as few nodes as possible until we've assigned all its cores
for (worker <- workers if worker.coresFree > 0) {
- for (job <- waitingJobs if job.coresLeft > 0) {
- if (canUse(job, worker)) {
- val coresToUse = math.min(worker.coresFree, job.coresLeft)
+ for (app <- waitingApps if app.coresLeft > 0) {
+ if (canUse(app, worker)) {
+ val coresToUse = math.min(worker.coresFree, app.coresLeft)
if (coresToUse > 0) {
- val exec = job.addExecutor(worker, coresToUse)
- launchExecutor(worker, exec, job.desc.sparkHome)
- job.state = JobState.RUNNING
+ val exec = app.addExecutor(worker, coresToUse)
+ launchExecutor(worker, exec, app.desc.sparkHome)
+ app.state = ApplicationState.RUNNING
}
}
}
@@ -208,8 +208,8 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo, sparkHome: String) {
logInfo("Launching executor " + exec.fullId + " on worker " + worker.id)
worker.addExecutor(exec)
- worker.actor ! LaunchExecutor(exec.job.id, exec.id, exec.job.desc, exec.cores, exec.memory, sparkHome)
- exec.job.driver ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory)
+ worker.actor ! LaunchExecutor(exec.application.id, exec.id, exec.application.desc, exec.cores, exec.memory, sparkHome)
+ exec.application.driver ! ExecutorAdded(exec.id, worker.id, worker.host, exec.cores, exec.memory)
}
def addWorker(id: String, host: String, port: Int, cores: Int, memory: Int, webUiPort: Int,
@@ -231,46 +231,46 @@ private[spark] class Master(ip: String, port: Int, webUiPort: Int) extends Actor
actorToWorker -= worker.actor
addressToWorker -= worker.actor.path.address
for (exec <- worker.executors.values) {
- logInfo("Telling job of lost executor: " + exec.id)
- exec.job.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None)
- exec.job.removeExecutor(exec)
+ logInfo("Telling app of lost executor: " + exec.id)
+ exec.application.driver ! ExecutorUpdated(exec.id, ExecutorState.LOST, Some("worker lost"), None)
+ exec.application.removeExecutor(exec)
}
}
- def addJob(desc: JobDescription, driver: ActorRef): JobInfo = {
+ def addApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = {
val now = System.currentTimeMillis()
val date = new Date(now)
- val job = new JobInfo(now, newJobId(date), desc, date, driver)
- jobs += job
- idToJob(job.id) = job
- actorToJob(driver) = job
- addressToJob(driver.path.address) = job
- return job
+ val app = new ApplicationInfo(now, newApplicationId(date), desc, date, driver)
+ apps += app
+ idToApp(app.id) = app
+ actorToApp(driver) = app
+ addressToApp(driver.path.address) = app
+ return app
}
- def removeJob(job: JobInfo) {
- if (jobs.contains(job)) {
- logInfo("Removing job " + job.id)
- jobs -= job
- idToJob -= job.id
- actorToJob -= job.driver
- addressToWorker -= job.driver.path.address
- completedJobs += job // Remember it in our history
- waitingJobs -= job
- for (exec <- job.executors.values) {
+ def removeApplication(app: ApplicationInfo) {
+ if (apps.contains(app)) {
+ logInfo("Removing app " + app.id)
+ apps -= app
+ idToApp -= app.id
+ actorToApp -= app.driver
+ addressToWorker -= app.driver.path.address
+ completedApps += app // Remember it in our history
+ waitingApps -= app
+ for (exec <- app.executors.values) {
exec.worker.removeExecutor(exec)
- exec.worker.actor ! KillExecutor(exec.job.id, exec.id)
+ exec.worker.actor ! KillExecutor(exec.application.id, exec.id)
}
- job.markFinished(JobState.FINISHED) // TODO: Mark it as FAILED if it failed
+ app.markFinished(ApplicationState.FINISHED) // TODO: Mark it as FAILED if it failed
schedule()
}
}
- /** Generate a new job ID given a job's submission date */
- def newJobId(submitDate: Date): String = {
- val jobId = "job-%s-%04d".format(DATE_FORMAT.format(submitDate), nextJobNumber)
- nextJobNumber += 1
- jobId
+ /** Generate a new app ID given a app's submission date */
+ def newApplicationId(submitDate: Date): String = {
+ val appId = "app-%s-%04d".format(DATE_FORMAT.format(submitDate), nextAppNumber)
+ nextAppNumber += 1
+ appId
}
/** Check for, and remove, any timed-out workers */
diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
index 529f72e9da..54faa375fb 100644
--- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
+++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala
@@ -40,27 +40,27 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct
}
}
} ~
- path("job") {
- parameters("jobId", 'format ?) {
- case (jobId, Some(js)) if (js.equalsIgnoreCase("json")) =>
+ path("app") {
+ parameters("appId", 'format ?) {
+ case (appId, Some(js)) if (js.equalsIgnoreCase("json")) =>
val future = master ? RequestMasterState
- val jobInfo = for (masterState <- future.mapTo[MasterState]) yield {
- masterState.activeJobs.find(_.id == jobId).getOrElse({
- masterState.completedJobs.find(_.id == jobId).getOrElse(null)
+ val appInfo = for (masterState <- future.mapTo[MasterState]) yield {
+ masterState.activeApps.find(_.id == appId).getOrElse({
+ masterState.completedApps.find(_.id == appId).getOrElse(null)
})
}
respondWithMediaType(MediaTypes.`application/json`) { ctx =>
- ctx.complete(jobInfo.mapTo[JobInfo])
+ ctx.complete(appInfo.mapTo[ApplicationInfo])
}
- case (jobId, _) =>
+ case (appId, _) =>
completeWith {
val future = master ? RequestMasterState
future.map { state =>
val masterState = state.asInstanceOf[MasterState]
- val job = masterState.activeJobs.find(_.id == jobId).getOrElse({
- masterState.completedJobs.find(_.id == jobId).getOrElse(null)
+ val app = masterState.activeApps.find(_.id == appId).getOrElse({
+ masterState.completedApps.find(_.id == appId).getOrElse(null)
})
- spark.deploy.master.html.job_details.render(job)
+ spark.deploy.master.html.app_details.render(app)
}
}
}
diff --git a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
index 2e467007a0..23df1bb463 100644
--- a/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
+++ b/core/src/main/scala/spark/deploy/master/WorkerInfo.scala
@@ -37,8 +37,8 @@ private[spark] class WorkerInfo(
}
}
- def hasExecutor(job: JobInfo): Boolean = {
- executors.values.exists(_.job == job)
+ def hasExecutor(app: ApplicationInfo): Boolean = {
+ executors.values.exists(_.application == app)
}
def webUiAddress : String = {
diff --git a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
index 4ef637090c..de11771c8e 100644
--- a/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
+++ b/core/src/main/scala/spark/deploy/worker/ExecutorRunner.scala
@@ -1,7 +1,7 @@
package spark.deploy.worker
import java.io._
-import spark.deploy.{ExecutorState, ExecutorStateChanged, JobDescription}
+import spark.deploy.{ExecutorState, ExecutorStateChanged, ApplicationDescription}
import akka.actor.ActorRef
import spark.{Utils, Logging}
import java.net.{URI, URL}
@@ -14,9 +14,9 @@ import spark.deploy.ExecutorStateChanged
* Manages the execution of one executor process.
*/
private[spark] class ExecutorRunner(
- val jobId: String,
+ val appId: String,
val execId: Int,
- val jobDesc: JobDescription,
+ val appDesc: ApplicationDescription,
val cores: Int,
val memory: Int,
val worker: ActorRef,
@@ -26,7 +26,7 @@ private[spark] class ExecutorRunner(
val workDir: File)
extends Logging {
- val fullId = jobId + "/" + execId
+ val fullId = appId + "/" + execId
var workerThread: Thread = null
var process: Process = null
var shutdownHook: Thread = null
@@ -60,7 +60,7 @@ private[spark] class ExecutorRunner(
process.destroy()
process.waitFor()
}
- worker ! ExecutorStateChanged(jobId, execId, ExecutorState.KILLED, None, None)
+ worker ! ExecutorStateChanged(appId, execId, ExecutorState.KILLED, None, None)
Runtime.getRuntime.removeShutdownHook(shutdownHook)
}
}
@@ -74,10 +74,10 @@ private[spark] class ExecutorRunner(
}
def buildCommandSeq(): Seq[String] = {
- val command = jobDesc.command
- val script = if (System.getProperty("os.name").startsWith("Windows")) "run.cmd" else "run";
+ val command = appDesc.command
+ 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)
+ Seq(runScript, command.mainClass) ++ (command.arguments ++ Seq(appId)).map(substituteVariables)
}
/** Spawn a thread that will redirect a given stream to a file */
@@ -96,12 +96,12 @@ private[spark] class ExecutorRunner(
}
/**
- * Download and run the executor described in our JobDescription
+ * Download and run the executor described in our ApplicationDescription
*/
def fetchAndRunExecutor() {
try {
// Create the executor's working directory
- val executorDir = new File(workDir, jobId + "/" + execId)
+ val executorDir = new File(workDir, appId + "/" + execId)
if (!executorDir.mkdirs()) {
throw new IOException("Failed to create directory " + executorDir)
}
@@ -110,7 +110,7 @@ private[spark] class ExecutorRunner(
val command = buildCommandSeq()
val builder = new ProcessBuilder(command: _*).directory(executorDir)
val env = builder.environment()
- for ((key, value) <- jobDesc.command.environment) {
+ for ((key, value) <- appDesc.command.environment) {
env.put(key, value)
}
env.put("SPARK_MEM", memory.toString + "m")
@@ -128,7 +128,7 @@ private[spark] class ExecutorRunner(
// times on the same machine.
val exitCode = process.waitFor()
val message = "Command exited with code " + exitCode
- worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message),
+ worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message),
Some(exitCode))
} catch {
case interrupted: InterruptedException =>
@@ -140,7 +140,7 @@ private[spark] class ExecutorRunner(
process.destroy()
}
val message = e.getClass + ": " + e.getMessage
- worker ! ExecutorStateChanged(jobId, execId, ExecutorState.FAILED, Some(message), None)
+ worker ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, Some(message), None)
}
}
}
diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala
index 924935a5fd..2bbc931316 100644
--- a/core/src/main/scala/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/spark/deploy/worker/Worker.scala
@@ -109,19 +109,19 @@ private[spark] class Worker(
logError("Worker registration failed: " + message)
System.exit(1)
- case LaunchExecutor(jobId, execId, jobDesc, cores_, memory_, execSparkHome_) =>
- logInfo("Asked to launch executor %s/%d for %s".format(jobId, execId, jobDesc.name))
+ case LaunchExecutor(appId, execId, appDesc, cores_, memory_, execSparkHome_) =>
+ logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
val manager = new ExecutorRunner(
- jobId, execId, jobDesc, cores_, memory_, self, workerId, ip, new File(execSparkHome_), workDir)
- executors(jobId + "/" + execId) = manager
+ appId, execId, appDesc, cores_, memory_, self, workerId, ip, new File(execSparkHome_), workDir)
+ executors(appId + "/" + execId) = manager
manager.start()
coresUsed += cores_
memoryUsed += memory_
- master ! ExecutorStateChanged(jobId, execId, ExecutorState.RUNNING, None, None)
+ master ! ExecutorStateChanged(appId, execId, ExecutorState.RUNNING, None, None)
- case ExecutorStateChanged(jobId, execId, state, message, exitStatus) =>
- master ! ExecutorStateChanged(jobId, execId, state, message, exitStatus)
- val fullId = jobId + "/" + execId
+ case ExecutorStateChanged(appId, execId, state, message, exitStatus) =>
+ master ! ExecutorStateChanged(appId, execId, state, message, exitStatus)
+ val fullId = appId + "/" + execId
if (ExecutorState.isFinished(state)) {
val executor = executors(fullId)
logInfo("Executor " + fullId + " finished with state " + state +
@@ -133,8 +133,8 @@ private[spark] class Worker(
memoryUsed -= executor.memory
}
- case KillExecutor(jobId, execId) =>
- val fullId = jobId + "/" + execId
+ case KillExecutor(appId, execId) =>
+ val fullId = appId + "/" + execId
executors.get(fullId) match {
case Some(executor) =>
logInfo("Asked to kill executor " + fullId)
diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
index 37524a7c82..08f02bad80 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala
@@ -92,7 +92,7 @@ private[spark] class WorkerArguments(args: Array[String]) {
"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" +
+ " -d DIR, --work-dir DIR Directory to run apps 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 ef81f072a3..135cc2e86c 100644
--- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
+++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala
@@ -41,9 +41,9 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct
}
} ~
path("log") {
- parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) =>
+ parameters("appId", "executorId", "logType") { (appId, executorId, logType) =>
respondWithMediaType(cc.spray.http.MediaTypes.`text/plain`) {
- getFromFileName("work/" + jobId + "/" + executorId + "/" + logType)
+ getFromFileName("work/" + appId + "/" + executorId + "/" + logType)
}
}
} ~
diff --git a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
index 224c126fdd..9a82c3054c 100644
--- a/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
+++ b/core/src/main/scala/spark/executor/StandaloneExecutorBackend.scala
@@ -68,8 +68,9 @@ private[spark] object StandaloneExecutorBackend {
}
def main(args: Array[String]) {
- if (args.length != 4) {
- System.err.println("Usage: StandaloneExecutorBackend <driverUrl> <executorId> <hostname> <cores>")
+ if (args.length < 4) {
+ //the reason we allow the last frameworkId argument is to make it easy to kill rogue executors
+ System.err.println("Usage: StandaloneExecutorBackend <driverUrl> <executorId> <hostname> <cores> [<appid>]")
System.exit(1)
}
run(args(0), args(1), args(2), args(3).toInt)
diff --git a/core/src/main/scala/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
index 24b4909380..de2dce161a 100644
--- a/core/src/main/scala/spark/partial/ApproximateActionListener.scala
+++ b/core/src/main/scala/spark/partial/ApproximateActionListener.scala
@@ -20,7 +20,7 @@ private[spark] class ApproximateActionListener[T, U, R](
extends JobListener {
val startTime = System.currentTimeMillis()
- val totalTasks = rdd.splits.size
+ val totalTasks = rdd.partitions.size
var finishedTasks = 0
var failure: Option[Exception] = None // Set if the job has failed (permanently)
var resultObject: Option[PartialResult[R]] = None // Set if we've already returned a PartialResult
diff --git a/core/src/main/scala/spark/rdd/BlockRDD.scala b/core/src/main/scala/spark/rdd/BlockRDD.scala
index 17989c5ce5..7348c4f15b 100644
--- a/core/src/main/scala/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/spark/rdd/BlockRDD.scala
@@ -1,9 +1,9 @@
package spark.rdd
import scala.collection.mutable.HashMap
-import spark.{RDD, SparkContext, SparkEnv, Split, TaskContext}
+import spark.{RDD, SparkContext, SparkEnv, Partition, TaskContext}
-private[spark] class BlockRDDSplit(val blockId: String, idx: Int) extends Split {
+private[spark] class BlockRDDPartition(val blockId: String, idx: Int) extends Partition {
val index = idx
}
@@ -18,14 +18,14 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
HashMap(blockIds.zip(locations):_*)
}
- override def getSplits: Array[Split] = (0 until blockIds.size).map(i => {
- new BlockRDDSplit(blockIds(i), i).asInstanceOf[Split]
+ override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => {
+ new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition]
}).toArray
- override def compute(split: Split, context: TaskContext): Iterator[T] = {
+ override def compute(split: Partition, context: TaskContext): Iterator[T] = {
val blockManager = SparkEnv.get.blockManager
- val blockId = split.asInstanceOf[BlockRDDSplit].blockId
+ val blockId = split.asInstanceOf[BlockRDDPartition].blockId
blockManager.get(blockId) match {
case Some(block) => block.asInstanceOf[Iterator[T]]
case None =>
@@ -33,8 +33,8 @@ class BlockRDD[T: ClassManifest](sc: SparkContext, @transient blockIds: Array[St
}
}
- override def getPreferredLocations(split: Split): Seq[String] =
- locations_(split.asInstanceOf[BlockRDDSplit].blockId)
+ override def getPreferredLocations(split: Partition): Seq[String] =
+ locations_(split.asInstanceOf[BlockRDDPartition].blockId)
}
diff --git a/core/src/main/scala/spark/rdd/CartesianRDD.scala b/core/src/main/scala/spark/rdd/CartesianRDD.scala
index 41cbbd0093..38600b8be4 100644
--- a/core/src/main/scala/spark/rdd/CartesianRDD.scala
+++ b/core/src/main/scala/spark/rdd/CartesianRDD.scala
@@ -5,22 +5,22 @@ import spark._
private[spark]
-class CartesianSplit(
+class CartesianPartition(
idx: Int,
@transient rdd1: RDD[_],
@transient rdd2: RDD[_],
s1Index: Int,
s2Index: Int
- ) extends Split {
- var s1 = rdd1.splits(s1Index)
- var s2 = rdd2.splits(s2Index)
+ ) extends Partition {
+ var s1 = rdd1.partitions(s1Index)
+ var s2 = rdd2.partitions(s2Index)
override val index: Int = idx
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
- s1 = rdd1.splits(s1Index)
- s2 = rdd2.splits(s2Index)
+ s1 = rdd1.partitions(s1Index)
+ s2 = rdd2.partitions(s2Index)
oos.defaultWriteObject()
}
}
@@ -33,35 +33,35 @@ class CartesianRDD[T: ClassManifest, U:ClassManifest](
extends RDD[Pair[T, U]](sc, Nil)
with Serializable {
- val numSplitsInRdd2 = rdd2.splits.size
+ val numPartitionsInRdd2 = rdd2.partitions.size
- override def getSplits: Array[Split] = {
+ override def getPartitions: Array[Partition] = {
// create the cross product split
- val array = new Array[Split](rdd1.splits.size * rdd2.splits.size)
- for (s1 <- rdd1.splits; s2 <- rdd2.splits) {
- val idx = s1.index * numSplitsInRdd2 + s2.index
- array(idx) = new CartesianSplit(idx, rdd1, rdd2, s1.index, s2.index)
+ val array = new Array[Partition](rdd1.partitions.size * rdd2.partitions.size)
+ for (s1 <- rdd1.partitions; s2 <- rdd2.partitions) {
+ val idx = s1.index * numPartitionsInRdd2 + s2.index
+ array(idx) = new CartesianPartition(idx, rdd1, rdd2, s1.index, s2.index)
}
array
}
- override def getPreferredLocations(split: Split): Seq[String] = {
- val currSplit = split.asInstanceOf[CartesianSplit]
+ override def getPreferredLocations(split: Partition): Seq[String] = {
+ val currSplit = split.asInstanceOf[CartesianPartition]
rdd1.preferredLocations(currSplit.s1) ++ rdd2.preferredLocations(currSplit.s2)
}
- override def compute(split: Split, context: TaskContext) = {
- val currSplit = split.asInstanceOf[CartesianSplit]
+ override def compute(split: Partition, context: TaskContext) = {
+ val currSplit = split.asInstanceOf[CartesianPartition]
for (x <- rdd1.iterator(currSplit.s1, context);
y <- rdd2.iterator(currSplit.s2, context)) yield (x, y)
}
override def getDependencies: Seq[Dependency[_]] = List(
new NarrowDependency(rdd1) {
- def getParents(id: Int): Seq[Int] = List(id / numSplitsInRdd2)
+ def getParents(id: Int): Seq[Int] = List(id / numPartitionsInRdd2)
},
new NarrowDependency(rdd2) {
- def getParents(id: Int): Seq[Int] = List(id % numSplitsInRdd2)
+ def getParents(id: Int): Seq[Int] = List(id % numPartitionsInRdd2)
}
)
diff --git a/core/src/main/scala/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
index 3558d4673f..36bfb0355e 100644
--- a/core/src/main/scala/spark/rdd/CheckpointRDD.scala
+++ b/core/src/main/scala/spark/rdd/CheckpointRDD.scala
@@ -9,7 +9,7 @@ import org.apache.hadoop.fs.Path
import java.io.{File, IOException, EOFException}
import java.text.NumberFormat
-private[spark] class CheckpointRDDSplit(val index: Int) extends Split {}
+private[spark] class CheckpointRDDPartition(val index: Int) extends Partition {}
/**
* This RDD represents a RDD checkpoint file (similar to HadoopRDD).
@@ -20,27 +20,27 @@ class CheckpointRDD[T: ClassManifest](sc: SparkContext, val checkpointPath: Stri
@transient val fs = new Path(checkpointPath).getFileSystem(sc.hadoopConfiguration)
- override def getSplits: Array[Split] = {
+ override def getPartitions: Array[Partition] = {
val dirContents = fs.listStatus(new Path(checkpointPath))
val splitFiles = dirContents.map(_.getPath.toString).filter(_.contains("part-")).sorted
- val numSplits = splitFiles.size
+ val numPartitions = splitFiles.size
if (!splitFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) ||
- !splitFiles(numSplits-1).endsWith(CheckpointRDD.splitIdToFile(numSplits-1))) {
+ !splitFiles(numPartitions-1).endsWith(CheckpointRDD.splitIdToFile(numPartitions-1))) {
throw new SparkException("Invalid checkpoint directory: " + checkpointPath)
}
- Array.tabulate(numSplits)(i => new CheckpointRDDSplit(i))
+ Array.tabulate(numPartitions)(i => new CheckpointRDDPartition(i))
}
checkpointData = Some(new RDDCheckpointData[T](this))
checkpointData.get.cpFile = Some(checkpointPath)
- override def getPreferredLocations(split: Split): Seq[String] = {
+ override def getPreferredLocations(split: Partition): Seq[String] = {
val status = fs.getFileStatus(new Path(checkpointPath))
val locations = fs.getFileBlockLocations(status, 0, status.getLen)
locations.headOption.toList.flatMap(_.getHosts).filter(_ != "localhost")
}
- override def compute(split: Split, context: TaskContext): Iterator[T] = {
+ override def compute(split: Partition, context: TaskContext): Iterator[T] = {
val file = new Path(checkpointPath, CheckpointRDD.splitIdToFile(split.index))
CheckpointRDD.readFromFile(file, context)
}
@@ -107,7 +107,7 @@ private[spark] object CheckpointRDD extends Logging {
deserializeStream.asIterator.asInstanceOf[Iterator[T]]
}
- // Test whether CheckpointRDD generate expected number of splits despite
+ // Test whether CheckpointRDD generate expected number of partitions despite
// each split file having multiple blocks. This needs to be run on a
// cluster (mesos or standalone) using HDFS.
def main(args: Array[String]) {
@@ -120,8 +120,8 @@ private[spark] object CheckpointRDD extends Logging {
val fs = path.getFileSystem(new Configuration())
sc.runJob(rdd, CheckpointRDD.writeToFile(path.toString, 1024) _)
val cpRDD = new CheckpointRDD[Int](sc, path.toString)
- assert(cpRDD.splits.length == rdd.splits.length, "Number of splits is not the same")
- assert(cpRDD.collect.toList == rdd.collect.toList, "Data of splits not the same")
+ assert(cpRDD.partitions.length == rdd.partitions.length, "Number of partitions is not the same")
+ assert(cpRDD.collect.toList == rdd.collect.toList, "Data of partitions not the same")
fs.delete(path)
}
}
diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
index 0a1e2cbee0..5200fb6b65 100644
--- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala
@@ -5,7 +5,7 @@ import java.util.{HashMap => JHashMap}
import scala.collection.JavaConversions
import scala.collection.mutable.ArrayBuffer
-import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Split, TaskContext}
+import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Partition, TaskContext}
import spark.{Dependency, OneToOneDependency, ShuffleDependency}
@@ -14,13 +14,13 @@ private[spark] sealed trait CoGroupSplitDep extends Serializable
private[spark] case class NarrowCoGroupSplitDep(
rdd: RDD[_],
splitIndex: Int,
- var split: Split
+ var split: Partition
) extends CoGroupSplitDep {
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
- split = rdd.splits(splitIndex)
+ split = rdd.partitions(splitIndex)
oos.defaultWriteObject()
}
}
@@ -28,7 +28,7 @@ private[spark] case class NarrowCoGroupSplitDep(
private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep
private[spark]
-class CoGroupSplit(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Split with Serializable {
+class CoGroupPartition(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Partition with Serializable {
override val index: Int = idx
override def hashCode(): Int = idx
}
@@ -40,8 +40,8 @@ private[spark] class CoGroupAggregator
{ (b1, b2) => b1 ++ b2 })
with Serializable
-class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner)
- extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) with Logging {
+class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner)
+ extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) {
private val aggr = new CoGroupAggregator
@@ -58,17 +58,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner)
}
}
- override def getSplits: Array[Split] = {
- val array = new Array[Split](part.numPartitions)
+ override def getPartitions: Array[Partition] = {
+ val array = new Array[Partition](part.numPartitions)
for (i <- 0 until array.size) {
- // Each CoGroupSplit will have a dependency per contributing RDD
- array(i) = new CoGroupSplit(i, rdds.zipWithIndex.map { case (rdd, j) =>
+ // Each CoGroupPartition will have a dependency per contributing RDD
+ array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) =>
// Assume each RDD contributed a single dependency, and get it
dependencies(j) match {
case s: ShuffleDependency[_, _] =>
new ShuffleCoGroupSplitDep(s.shuffleId)
case _ =>
- new NarrowCoGroupSplitDep(rdd, i, rdd.splits(i))
+ new NarrowCoGroupSplitDep(rdd, i, rdd.partitions(i))
}
}.toList)
}
@@ -77,8 +77,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(_, _)]], part: Partitioner)
override val partitioner = Some(part)
- override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = {
- val split = s.asInstanceOf[CoGroupSplit]
+ override def compute(s: Partition, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = {
+ val split = s.asInstanceOf[CoGroupPartition]
val numRdds = split.deps.size
// e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs)
val map = new JHashMap[K, Seq[ArrayBuffer[Any]]]
diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
index fcd26da43a..0d16cf6e85 100644
--- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala
+++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala
@@ -1,19 +1,19 @@
package spark.rdd
-import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Split, TaskContext}
+import spark.{Dependency, OneToOneDependency, NarrowDependency, RDD, Partition, TaskContext}
import java.io.{ObjectOutputStream, IOException}
-private[spark] case class CoalescedRDDSplit(
+private[spark] case class CoalescedRDDPartition(
index: Int,
@transient rdd: RDD[_],
parentsIndices: Array[Int]
- ) extends Split {
- var parents: Seq[Split] = parentsIndices.map(rdd.splits(_))
+ ) extends Partition {
+ var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_))
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
- parents = parentsIndices.map(rdd.splits(_))
+ parents = parentsIndices.map(rdd.partitions(_))
oos.defaultWriteObject()
}
}
@@ -31,21 +31,21 @@ class CoalescedRDD[T: ClassManifest](
maxPartitions: Int)
extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies
- override def getSplits: Array[Split] = {
- val prevSplits = prev.splits
+ override def getPartitions: Array[Partition] = {
+ val prevSplits = prev.partitions
if (prevSplits.length < maxPartitions) {
- prevSplits.map(_.index).map{idx => new CoalescedRDDSplit(idx, prev, Array(idx)) }
+ prevSplits.map(_.index).map{idx => new CoalescedRDDPartition(idx, prev, Array(idx)) }
} else {
(0 until maxPartitions).map { i =>
val rangeStart = (i * prevSplits.length) / maxPartitions
val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions
- new CoalescedRDDSplit(i, prev, (rangeStart until rangeEnd).toArray)
+ new CoalescedRDDPartition(i, prev, (rangeStart until rangeEnd).toArray)
}.toArray
}
}
- override def compute(split: Split, context: TaskContext): Iterator[T] = {
- split.asInstanceOf[CoalescedRDDSplit].parents.iterator.flatMap { parentSplit =>
+ override def compute(split: Partition, context: TaskContext): Iterator[T] = {
+ split.asInstanceOf[CoalescedRDDPartition].parents.iterator.flatMap { parentSplit =>
firstParent[T].iterator(parentSplit, context)
}
}
@@ -53,7 +53,7 @@ class CoalescedRDD[T: ClassManifest](
override def getDependencies: Seq[Dependency[_]] = {
Seq(new NarrowDependency(prev) {
def getParents(id: Int): Seq[Int] =
- splits(id).asInstanceOf[CoalescedRDDSplit].parentsIndices
+ partitions(id).asInstanceOf[CoalescedRDDPartition].parentsIndices
})
}
diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala
index 93e398ea2b..c84ec39d21 100644
--- a/core/src/main/scala/spark/rdd/FilteredRDD.scala
+++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala
@@ -1,16 +1,16 @@
package spark.rdd
-import spark.{OneToOneDependency, RDD, Split, TaskContext}
+import spark.{OneToOneDependency, RDD, Partition, TaskContext}
private[spark] class FilteredRDD[T: ClassManifest](
prev: RDD[T],
f: T => Boolean)
extends RDD[T](prev) {
- override def getSplits: Array[Split] = firstParent[T].splits
+ override def getPartitions: Array[Partition] = firstParent[T].partitions
override val partitioner = prev.partitioner // Since filter cannot change a partition's keys
- override def compute(split: Split, context: TaskContext) =
+ override def compute(split: Partition, context: TaskContext) =
firstParent[T].iterator(split, context).filter(f)
}
diff --git a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
index 8c2a610593..8ebc778925 100644
--- a/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/FlatMappedRDD.scala
@@ -1,6 +1,6 @@
package spark.rdd
-import spark.{RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
private[spark]
@@ -9,8 +9,8 @@ class FlatMappedRDD[U: ClassManifest, T: ClassManifest](
f: T => TraversableOnce[U])
extends RDD[U](prev) {
- override def getSplits: Array[Split] = firstParent[T].splits
+ override def getPartitions: Array[Partition] = firstParent[T].partitions
- override def compute(split: Split, context: TaskContext) =
+ override def compute(split: Partition, context: TaskContext) =
firstParent[T].iterator(split, context).flatMap(f)
}
diff --git a/core/src/main/scala/spark/rdd/GlommedRDD.scala b/core/src/main/scala/spark/rdd/GlommedRDD.scala
index 70b9b4e34e..e16c7ba881 100644
--- a/core/src/main/scala/spark/rdd/GlommedRDD.scala
+++ b/core/src/main/scala/spark/rdd/GlommedRDD.scala
@@ -1,12 +1,12 @@
package spark.rdd
-import spark.{RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
private[spark] class GlommedRDD[T: ClassManifest](prev: RDD[T])
extends RDD[Array[T]](prev) {
- override def getSplits: Array[Split] = firstParent[T].splits
+ override def getPartitions: Array[Partition] = firstParent[T].partitions
- override def compute(split: Split, context: TaskContext) =
+ override def compute(split: Partition, context: TaskContext) =
Array(firstParent[T].iterator(split, context).toArray).iterator
}
diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala
index 854993737b..8139a2a40c 100644
--- a/core/src/main/scala/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala
@@ -15,14 +15,14 @@ import org.apache.hadoop.mapred.RecordReader
import org.apache.hadoop.mapred.Reporter
import org.apache.hadoop.util.ReflectionUtils
-import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext}
+import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext}
/**
* A Spark split class that wraps around a Hadoop InputSplit.
*/
-private[spark] class HadoopSplit(rddId: Int, idx: Int, @transient s: InputSplit)
- extends Split {
+private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSplit)
+ extends Partition {
val inputSplit = new SerializableWritable[InputSplit](s)
@@ -47,12 +47,12 @@ class HadoopRDD[K, V](
// A Hadoop JobConf can be about 10 KB, which is pretty big, so broadcast it
private val confBroadcast = sc.broadcast(new SerializableWritable(conf))
- override def getSplits: Array[Split] = {
+ override def getPartitions: Array[Partition] = {
val inputFormat = createInputFormat(conf)
val inputSplits = inputFormat.getSplits(conf, minSplits)
- val array = new Array[Split](inputSplits.size)
+ val array = new Array[Partition](inputSplits.size)
for (i <- 0 until inputSplits.size) {
- array(i) = new HadoopSplit(id, i, inputSplits(i))
+ array(i) = new HadoopPartition(id, i, inputSplits(i))
}
array
}
@@ -62,8 +62,8 @@ class HadoopRDD[K, V](
.asInstanceOf[InputFormat[K, V]]
}
- override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
- val split = theSplit.asInstanceOf[HadoopSplit]
+ override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] {
+ val split = theSplit.asInstanceOf[HadoopPartition]
var reader: RecordReader[K, V] = null
val conf = confBroadcast.value.value
@@ -106,9 +106,9 @@ class HadoopRDD[K, V](
}
}
- override def getPreferredLocations(split: Split): Seq[String] = {
+ override def getPreferredLocations(split: Partition): Seq[String] = {
// TODO: Filtering out "localhost" in case of file:// URLs
- val hadoopSplit = split.asInstanceOf[HadoopSplit]
+ val hadoopSplit = split.asInstanceOf[HadoopPartition]
hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost")
}
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
index 7b0b4525c7..d283c5b2bb 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/spark/rdd/MapPartitionsRDD.scala
@@ -1,6 +1,6 @@
package spark.rdd
-import spark.{RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
private[spark]
@@ -13,8 +13,8 @@ class MapPartitionsRDD[U: ClassManifest, T: ClassManifest](
override val partitioner =
if (preservesPartitioning) firstParent[T].partitioner else None
- override def getSplits: Array[Split] = firstParent[T].splits
+ override def getPartitions: Array[Partition] = firstParent[T].partitions
- override def compute(split: Split, context: TaskContext) =
+ override def compute(split: Partition, context: TaskContext) =
f(firstParent[T].iterator(split, context))
-} \ No newline at end of file
+}
diff --git a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
index c6dc1080a9..afb7504ba1 100644
--- a/core/src/main/scala/spark/rdd/MapPartitionsWithSplitRDD.scala
+++ b/core/src/main/scala/spark/rdd/MapPartitionsWithIndexRDD.scala
@@ -1,24 +1,24 @@
package spark.rdd
-import spark.{RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
/**
- * A variant of the MapPartitionsRDD that passes the split index into the
+ * A variant of the MapPartitionsRDD that passes the partition index into the
* closure. This can be used to generate or collect partition specific
* information such as the number of tuples in a partition.
*/
private[spark]
-class MapPartitionsWithSplitRDD[U: ClassManifest, T: ClassManifest](
+class MapPartitionsWithIndexRDD[U: ClassManifest, T: ClassManifest](
prev: RDD[T],
f: (Int, Iterator[T]) => Iterator[U],
preservesPartitioning: Boolean
) extends RDD[U](prev) {
- override def getSplits: Array[Split] = firstParent[T].splits
+ override def getPartitions: Array[Partition] = firstParent[T].partitions
override val partitioner = if (preservesPartitioning) prev.partitioner else None
- override def compute(split: Split, context: TaskContext) =
+ override def compute(split: Partition, context: TaskContext) =
f(split.index, firstParent[T].iterator(split, context))
-} \ 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
index 6074f411e3..af07311b6d 100644
--- a/core/src/main/scala/spark/rdd/MappedRDD.scala
+++ b/core/src/main/scala/spark/rdd/MappedRDD.scala
@@ -1,13 +1,13 @@
package spark.rdd
-import spark.{RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
private[spark]
class MappedRDD[U: ClassManifest, T: ClassManifest](prev: RDD[T], f: T => U)
extends RDD[U](prev) {
- override def getSplits: Array[Split] = firstParent[T].splits
+ override def getPartitions: Array[Partition] = firstParent[T].partitions
- override def compute(split: Split, context: TaskContext) =
+ override def compute(split: Partition, context: TaskContext) =
firstParent[T].iterator(split, context).map(f)
}
diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
index 345ae79d74..ebd4c3f0e2 100644
--- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala
@@ -7,12 +7,12 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapreduce._
-import spark.{Dependency, RDD, SerializableWritable, SparkContext, Split, TaskContext}
+import spark.{Dependency, RDD, SerializableWritable, SparkContext, Partition, TaskContext}
private[spark]
-class NewHadoopSplit(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable)
- extends Split {
+class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable)
+ extends Partition {
val serializableHadoopSplit = new SerializableWritable(rawSplit)
@@ -39,19 +39,19 @@ class NewHadoopRDD[K, V](
@transient private val jobId = new JobID(jobtrackerId, id)
- override def getSplits: Array[Split] = {
+ override def getPartitions: Array[Partition] = {
val inputFormat = inputFormatClass.newInstance
val jobContext = newJobContext(conf, jobId)
val rawSplits = inputFormat.getSplits(jobContext).toArray
- val result = new Array[Split](rawSplits.size)
+ val result = new Array[Partition](rawSplits.size)
for (i <- 0 until rawSplits.size) {
- result(i) = new NewHadoopSplit(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
+ result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
}
result
}
- override def compute(theSplit: Split, context: TaskContext) = new Iterator[(K, V)] {
- val split = theSplit.asInstanceOf[NewHadoopSplit]
+ override def compute(theSplit: Partition, context: TaskContext) = new Iterator[(K, V)] {
+ val split = theSplit.asInstanceOf[NewHadoopPartition]
val conf = confBroadcast.value.value
val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0)
val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId)
@@ -83,8 +83,8 @@ class NewHadoopRDD[K, V](
}
}
- override def getPreferredLocations(split: Split): Seq[String] = {
- val theSplit = split.asInstanceOf[NewHadoopSplit]
+ override def getPreferredLocations(split: Partition): Seq[String] = {
+ val theSplit = split.asInstanceOf[NewHadoopPartition]
theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost")
}
}
diff --git a/core/src/main/scala/spark/ParallelCollection.scala b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
index 10adcd53ec..07585a88ce 100644
--- a/core/src/main/scala/spark/ParallelCollection.scala
+++ b/core/src/main/scala/spark/rdd/ParallelCollectionRDD.scala
@@ -1,28 +1,29 @@
-package spark
+package spark.rdd
import scala.collection.immutable.NumericRange
import scala.collection.mutable.ArrayBuffer
import scala.collection.Map
+import spark.{RDD, TaskContext, SparkContext, Partition}
-private[spark] class ParallelCollectionSplit[T: ClassManifest](
+private[spark] class ParallelCollectionPartition[T: ClassManifest](
val rddId: Long,
val slice: Int,
values: Seq[T])
- extends Split with Serializable {
+ extends Partition with Serializable {
def iterator: Iterator[T] = values.iterator
override def hashCode(): Int = (41 * (41 + rddId) + slice).toInt
override def equals(other: Any): Boolean = other match {
- case that: ParallelCollectionSplit[_] => (this.rddId == that.rddId && this.slice == that.slice)
+ case that: ParallelCollectionPartition[_] => (this.rddId == that.rddId && this.slice == that.slice)
case _ => false
}
override val index: Int = slice
}
-private[spark] class ParallelCollection[T: ClassManifest](
+private[spark] class ParallelCollectionRDD[T: ClassManifest](
@transient sc: SparkContext,
@transient data: Seq[T],
numSlices: Int,
@@ -33,26 +34,20 @@ private[spark] class ParallelCollection[T: ClassManifest](
// instead.
// UPDATE: A parallel collection can be checkpointed to HDFS, which achieves this goal.
- @transient var splits_ : Array[Split] = {
- val slices = ParallelCollection.slice(data, numSlices).toArray
- slices.indices.map(i => new ParallelCollectionSplit(id, i, slices(i))).toArray
+ override def getPartitions: Array[Partition] = {
+ val slices = ParallelCollectionRDD.slice(data, numSlices).toArray
+ slices.indices.map(i => new ParallelCollectionPartition(id, i, slices(i))).toArray
}
- override def getSplits = splits_
+ override def compute(s: Partition, context: TaskContext) =
+ s.asInstanceOf[ParallelCollectionPartition[T]].iterator
- override def compute(s: Split, context: TaskContext) =
- s.asInstanceOf[ParallelCollectionSplit[T]].iterator
-
- override def getPreferredLocations(s: Split): Seq[String] = {
+ override def getPreferredLocations(s: Partition): Seq[String] = {
locationPrefs.getOrElse(s.index, Nil)
}
-
- override def clearDependencies() {
- splits_ = null
- }
}
-private object ParallelCollection {
+private object ParallelCollectionRDD {
/**
* Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range
* collections specially, encoding the slices as other Ranges to minimize memory cost. This makes
diff --git a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
index d1553181c1..f2f4fd56d1 100644
--- a/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
+++ b/core/src/main/scala/spark/rdd/PartitionPruningRDD.scala
@@ -1,9 +1,9 @@
package spark.rdd
-import spark.{NarrowDependency, RDD, SparkEnv, Split, TaskContext}
+import spark.{NarrowDependency, RDD, SparkEnv, Partition, TaskContext}
-class PartitionPruningRDDSplit(idx: Int, val parentSplit: Split) extends Split {
+class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition {
override val index = idx
}
@@ -16,15 +16,15 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo
extends NarrowDependency[T](rdd) {
@transient
- val partitions: Array[Split] = rdd.splits.filter(s => partitionFilterFunc(s.index))
- .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDSplit(idx, split) : Split }
+ val partitions: Array[Partition] = rdd.partitions.filter(s => partitionFilterFunc(s.index))
+ .zipWithIndex.map { case(split, idx) => new PartitionPruningRDDPartition(idx, split) : Partition }
override def getParents(partitionId: Int) = List(partitions(partitionId).index)
}
/**
- * A RDD used to prune RDD partitions/splits so we can avoid launching tasks on
+ * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on
* all partitions. An example use case: If we know the RDD is partitioned by range,
* and the execution DAG has a filter on the key, we can avoid launching tasks
* on partitions that don't have the range covering the key.
@@ -34,9 +34,9 @@ class PartitionPruningRDD[T: ClassManifest](
@transient partitionFilterFunc: Int => Boolean)
extends RDD[T](prev.context, List(new PruneDependency(prev, partitionFilterFunc))) {
- override def compute(split: Split, context: TaskContext) = firstParent[T].iterator(
- split.asInstanceOf[PartitionPruningRDDSplit].parentSplit, context)
+ override def compute(split: Partition, context: TaskContext) = firstParent[T].iterator(
+ split.asInstanceOf[PartitionPruningRDDPartition].parentSplit, context)
- override protected def getSplits: Array[Split] =
+ override protected def getPartitions: Array[Partition] =
getDependencies.head.asInstanceOf[PruneDependency[T]].partitions
}
diff --git a/core/src/main/scala/spark/rdd/PipedRDD.scala b/core/src/main/scala/spark/rdd/PipedRDD.scala
index 56032a8659..962a1b21ad 100644
--- a/core/src/main/scala/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/spark/rdd/PipedRDD.scala
@@ -8,7 +8,7 @@ import scala.collection.JavaConversions._
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
-import spark.{RDD, SparkEnv, Split, TaskContext}
+import spark.{RDD, SparkEnv, Partition, TaskContext}
/**
@@ -27,9 +27,9 @@ class PipedRDD[T: ClassManifest](
// using a standard StringTokenizer (i.e. by spaces)
def this(prev: RDD[T], command: String) = this(prev, PipedRDD.tokenize(command))
- override def getSplits: Array[Split] = firstParent[T].splits
+ override def getPartitions: Array[Partition] = firstParent[T].partitions
- override def compute(split: Split, context: TaskContext): Iterator[String] = {
+ override def compute(split: Partition, context: TaskContext): Iterator[String] = {
val pb = new ProcessBuilder(command)
// Add the environmental variables to the process.
val currentEnvVars = pb.environment()
diff --git a/core/src/main/scala/spark/rdd/SampledRDD.scala b/core/src/main/scala/spark/rdd/SampledRDD.scala
index f2a144e2e0..243673f151 100644
--- a/core/src/main/scala/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/spark/rdd/SampledRDD.scala
@@ -5,10 +5,10 @@ import java.util.Random
import cern.jet.random.Poisson
import cern.jet.random.engine.DRand
-import spark.{RDD, Split, TaskContext}
+import spark.{RDD, Partition, TaskContext}
private[spark]
-class SampledRDDSplit(val prev: Split, val seed: Int) extends Split with Serializable {
+class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable {
override val index: Int = prev.index
}
@@ -19,16 +19,16 @@ class SampledRDD[T: ClassManifest](
seed: Int)
extends RDD[T](prev) {
- override def getSplits: Array[Split] = {
+ override def getPartitions: Array[Partition] = {
val rg = new Random(seed)
- firstParent[T].splits.map(x => new SampledRDDSplit(x, rg.nextInt))
+ firstParent[T].partitions.map(x => new SampledRDDPartition(x, rg.nextInt))
}
- override def getPreferredLocations(split: Split): Seq[String] =
- firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDSplit].prev)
+ override def getPreferredLocations(split: Partition): Seq[String] =
+ firstParent[T].preferredLocations(split.asInstanceOf[SampledRDDPartition].prev)
- override def compute(splitIn: Split, context: TaskContext): Iterator[T] = {
- val split = splitIn.asInstanceOf[SampledRDDSplit]
+ override def compute(splitIn: Partition, context: TaskContext): Iterator[T] = {
+ val split = splitIn.asInstanceOf[SampledRDDPartition]
if (withReplacement) {
// 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.
diff --git a/core/src/main/scala/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
index bf69b5150b..c2f118305f 100644
--- a/core/src/main/scala/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/spark/rdd/ShuffledRDD.scala
@@ -1,9 +1,9 @@
package spark.rdd
-import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Split, TaskContext}
+import spark.{Partitioner, RDD, SparkEnv, ShuffleDependency, Partition, TaskContext}
import spark.SparkContext._
-private[spark] class ShuffledRDDSplit(val idx: Int) extends Split {
+private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition {
override val index = idx
override def hashCode(): Int = idx
}
@@ -22,11 +22,11 @@ class ShuffledRDD[K, V](
override val partitioner = Some(part)
- override def getSplits: Array[Split] = {
- Array.tabulate[Split](part.numPartitions)(i => new ShuffledRDDSplit(i))
+ override def getPartitions: Array[Partition] = {
+ Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i))
}
- override def compute(split: Split, context: TaskContext): Iterator[(K, V)] = {
+ override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId
SparkEnv.get.shuffleFetcher.fetch[K, V](shuffledId, split.index)
}
diff --git a/core/src/main/scala/spark/rdd/UnionRDD.scala b/core/src/main/scala/spark/rdd/UnionRDD.scala
index ebc0068228..2c52a67e22 100644
--- a/core/src/main/scala/spark/rdd/UnionRDD.scala
+++ b/core/src/main/scala/spark/rdd/UnionRDD.scala
@@ -1,13 +1,13 @@
package spark.rdd
import scala.collection.mutable.ArrayBuffer
-import spark.{Dependency, RangeDependency, RDD, SparkContext, Split, TaskContext}
+import spark.{Dependency, RangeDependency, RDD, SparkContext, Partition, TaskContext}
import java.io.{ObjectOutputStream, IOException}
-private[spark] class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
- extends Split {
+private[spark] class UnionPartition[T: ClassManifest](idx: Int, rdd: RDD[T], splitIndex: Int)
+ extends Partition {
- var split: Split = rdd.splits(splitIndex)
+ var split: Partition = rdd.partitions(splitIndex)
def iterator(context: TaskContext) = rdd.iterator(split, context)
@@ -18,7 +18,7 @@ private[spark] class UnionSplit[T: ClassManifest](idx: Int, rdd: RDD[T], splitIn
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
- split = rdd.splits(splitIndex)
+ split = rdd.partitions(splitIndex)
oos.defaultWriteObject()
}
}
@@ -28,11 +28,11 @@ class UnionRDD[T: ClassManifest](
@transient var rdds: Seq[RDD[T]])
extends RDD[T](sc, Nil) { // Nil since we implement getDependencies
- override def getSplits: Array[Split] = {
- val array = new Array[Split](rdds.map(_.splits.size).sum)
+ override def getPartitions: Array[Partition] = {
+ val array = new Array[Partition](rdds.map(_.partitions.size).sum)
var pos = 0
- for (rdd <- rdds; split <- rdd.splits) {
- array(pos) = new UnionSplit(pos, rdd, split.index)
+ for (rdd <- rdds; split <- rdd.partitions) {
+ array(pos) = new UnionPartition(pos, rdd, split.index)
pos += 1
}
array
@@ -42,15 +42,15 @@ class UnionRDD[T: ClassManifest](
val deps = new ArrayBuffer[Dependency[_]]
var pos = 0
for (rdd <- rdds) {
- deps += new RangeDependency(rdd, 0, pos, rdd.splits.size)
- pos += rdd.splits.size
+ deps += new RangeDependency(rdd, 0, pos, rdd.partitions.size)
+ pos += rdd.partitions.size
}
deps
}
- override def compute(s: Split, context: TaskContext): Iterator[T] =
- s.asInstanceOf[UnionSplit[T]].iterator(context)
+ override def compute(s: Partition, context: TaskContext): Iterator[T] =
+ s.asInstanceOf[UnionPartition[T]].iterator(context)
- override def getPreferredLocations(s: Split): Seq[String] =
- s.asInstanceOf[UnionSplit[T]].preferredLocations()
+ override def getPreferredLocations(s: Partition): Seq[String] =
+ s.asInstanceOf[UnionPartition[T]].preferredLocations()
}
diff --git a/core/src/main/scala/spark/rdd/ZippedRDD.scala b/core/src/main/scala/spark/rdd/ZippedRDD.scala
index 1ce70268bb..e80ec17aa5 100644
--- a/core/src/main/scala/spark/rdd/ZippedRDD.scala
+++ b/core/src/main/scala/spark/rdd/ZippedRDD.scala
@@ -1,17 +1,17 @@
package spark.rdd
-import spark.{OneToOneDependency, RDD, SparkContext, Split, TaskContext}
+import spark.{OneToOneDependency, RDD, SparkContext, Partition, TaskContext}
import java.io.{ObjectOutputStream, IOException}
-private[spark] class ZippedSplit[T: ClassManifest, U: ClassManifest](
+private[spark] class ZippedPartition[T: ClassManifest, U: ClassManifest](
idx: Int,
@transient rdd1: RDD[T],
@transient rdd2: RDD[U]
- ) extends Split {
+ ) extends Partition {
- var split1 = rdd1.splits(idx)
- var split2 = rdd1.splits(idx)
+ var split1 = rdd1.partitions(idx)
+ var split2 = rdd1.partitions(idx)
override val index: Int = idx
def splits = (split1, split2)
@@ -19,8 +19,8 @@ private[spark] class ZippedSplit[T: ClassManifest, U: ClassManifest](
@throws(classOf[IOException])
private def writeObject(oos: ObjectOutputStream) {
// Update the reference to parent split at the time of task serialization
- split1 = rdd1.splits(idx)
- split2 = rdd2.splits(idx)
+ split1 = rdd1.partitions(idx)
+ split2 = rdd2.partitions(idx)
oos.defaultWriteObject()
}
}
@@ -31,24 +31,24 @@ class ZippedRDD[T: ClassManifest, U: ClassManifest](
var rdd2: RDD[U])
extends RDD[(T, U)](sc, List(new OneToOneDependency(rdd1), new OneToOneDependency(rdd2))) {
- override def getSplits: Array[Split] = {
- if (rdd1.splits.size != rdd2.splits.size) {
+ override def getPartitions: Array[Partition] = {
+ if (rdd1.partitions.size != rdd2.partitions.size) {
throw new IllegalArgumentException("Can't zip RDDs with unequal numbers of partitions")
}
- val array = new Array[Split](rdd1.splits.size)
- for (i <- 0 until rdd1.splits.size) {
- array(i) = new ZippedSplit(i, rdd1, rdd2)
+ val array = new Array[Partition](rdd1.partitions.size)
+ for (i <- 0 until rdd1.partitions.size) {
+ array(i) = new ZippedPartition(i, rdd1, rdd2)
}
array
}
- override def compute(s: Split, context: TaskContext): Iterator[(T, U)] = {
- val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits
+ override def compute(s: Partition, context: TaskContext): Iterator[(T, U)] = {
+ val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits
rdd1.iterator(split1, context).zip(rdd2.iterator(split2, context))
}
- override def getPreferredLocations(s: Split): Seq[String] = {
- val (split1, split2) = s.asInstanceOf[ZippedSplit[T, U]].splits
+ override def getPreferredLocations(s: Partition): Seq[String] = {
+ val (split1, split2) = s.asInstanceOf[ZippedPartition[T, U]].splits
rdd1.preferredLocations(split1).intersect(rdd2.preferredLocations(split2))
}
diff --git a/core/src/main/scala/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
index 319eef6978..bf0837c066 100644
--- a/core/src/main/scala/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/spark/scheduler/DAGScheduler.scala
@@ -106,7 +106,7 @@ class DAGScheduler(
private def getCacheLocs(rdd: RDD[_]): Array[List[String]] = {
if (!cacheLocs.contains(rdd.id)) {
- val blockIds = rdd.splits.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray
+ val blockIds = rdd.partitions.indices.map(index=> "rdd_%d_%d".format(rdd.id, index)).toArray
cacheLocs(rdd.id) = blockManagerMaster.getLocations(blockIds).map {
locations => locations.map(_.ip).toList
}.toArray
@@ -141,9 +141,9 @@ class DAGScheduler(
private def newStage(rdd: RDD[_], shuffleDep: Option[ShuffleDependency[_,_]], priority: Int): Stage = {
if (shuffleDep != None) {
// Kind of ugly: need to register RDDs with the cache and map output tracker here
- // since we can't do it in the RDD constructor because # of splits is unknown
+ // since we can't do it in the RDD constructor because # of partitions is unknown
logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")")
- mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.splits.size)
+ mapOutputTracker.registerShuffle(shuffleDep.get.shuffleId, rdd.partitions.size)
}
val id = nextStageId.getAndIncrement()
val stage = new Stage(id, rdd, shuffleDep, getParentStages(rdd, priority), priority)
@@ -162,7 +162,7 @@ class DAGScheduler(
if (!visited(r)) {
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
+ // we can't do it in its constructor because # of partitions is unknown
for (dep <- r.dependencies) {
dep match {
case shufDep: ShuffleDependency[_,_] =>
@@ -257,7 +257,7 @@ class DAGScheduler(
{
val listener = new ApproximateActionListener(rdd, func, evaluator, timeout)
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
- val partitions = (0 until rdd.splits.size).toArray
+ val partitions = (0 until rdd.partitions.size).toArray
eventQueue.put(JobSubmitted(rdd, func2, partitions, false, callSite, listener))
return listener.awaitResult() // Will throw an exception if the job fails
}
@@ -386,7 +386,7 @@ class DAGScheduler(
try {
SparkEnv.set(env)
val rdd = job.finalStage.rdd
- val split = rdd.splits(job.partitions(0))
+ val split = rdd.partitions(job.partitions(0))
val taskContext = new TaskContext(job.finalStage.id, job.partitions(0), 0)
try {
val result = job.func(taskContext, rdd.iterator(split, taskContext))
@@ -672,7 +672,7 @@ class DAGScheduler(
return cached
}
// If the RDD has some placement preferences (as is the case for input RDDs), get those
- val rddPrefs = rdd.preferredLocations(rdd.splits(partition)).toList
+ val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList
if (rddPrefs != Nil) {
return rddPrefs
}
diff --git a/core/src/main/scala/spark/scheduler/ResultTask.scala b/core/src/main/scala/spark/scheduler/ResultTask.scala
index 8cd4c661eb..1721f78f48 100644
--- a/core/src/main/scala/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/spark/scheduler/ResultTask.scala
@@ -67,7 +67,7 @@ private[spark] class ResultTask[T, U](
var split = if (rdd == null) {
null
} else {
- rdd.splits(partition)
+ rdd.partitions(partition)
}
override def run(attemptId: Long): U = {
@@ -85,7 +85,7 @@ private[spark] class ResultTask[T, U](
override def writeExternal(out: ObjectOutput) {
RDDCheckpointData.synchronized {
- split = rdd.splits(partition)
+ split = rdd.partitions(partition)
out.writeInt(stageId)
val bytes = ResultTask.serializeInfo(
stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _])
@@ -107,6 +107,6 @@ private[spark] class ResultTask[T, U](
func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U]
partition = in.readInt()
val outputId = in.readInt()
- split = in.readObject().asInstanceOf[Split]
+ split = in.readObject().asInstanceOf[Partition]
}
}
diff --git a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
index bed9f1864f..59ee3c0a09 100644
--- a/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/spark/scheduler/ShuffleMapTask.scala
@@ -86,12 +86,12 @@ private[spark] class ShuffleMapTask(
var split = if (rdd == null) {
null
} else {
- rdd.splits(partition)
+ rdd.partitions(partition)
}
override def writeExternal(out: ObjectOutput) {
RDDCheckpointData.synchronized {
- split = rdd.splits(partition)
+ split = rdd.partitions(partition)
out.writeInt(stageId)
val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep)
out.writeInt(bytes.length)
@@ -112,7 +112,7 @@ private[spark] class ShuffleMapTask(
dep = dep_
partition = in.readInt()
generation = in.readLong()
- split = in.readObject().asInstanceOf[Split]
+ split = in.readObject().asInstanceOf[Partition]
}
override def run(attemptId: Long): MapStatus = {
diff --git a/core/src/main/scala/spark/scheduler/Stage.scala b/core/src/main/scala/spark/scheduler/Stage.scala
index 374114d870..552061e46b 100644
--- a/core/src/main/scala/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/spark/scheduler/Stage.scala
@@ -28,7 +28,7 @@ private[spark] class Stage(
extends Logging {
val isShuffleMap = shuffleDep != None
- val numPartitions = rdd.splits.size
+ val numPartitions = rdd.partitions.size
val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil)
var numAvailableOutputs = 0
diff --git a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index e77355c6cd..bb289c9cf3 100644
--- a/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
@@ -2,14 +2,14 @@ package spark.scheduler.cluster
import spark.{Utils, Logging, SparkContext}
import spark.deploy.client.{Client, ClientListener}
-import spark.deploy.{Command, JobDescription}
+import spark.deploy.{Command, ApplicationDescription}
import scala.collection.mutable.HashMap
private[spark] class SparkDeploySchedulerBackend(
scheduler: ClusterScheduler,
sc: SparkContext,
master: String,
- jobName: String)
+ appName: String)
extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
with ClientListener
with Logging {
@@ -29,10 +29,11 @@ private[spark] class SparkDeploySchedulerBackend(
StandaloneSchedulerBackend.ACTOR_NAME)
val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}")
val command = Command("spark.executor.StandaloneExecutorBackend", args, sc.executorEnvs)
- val sparkHome = sc.getSparkHome().getOrElse(throw new IllegalArgumentException("must supply spark home for spark standalone"))
- val jobDesc = new JobDescription(jobName, maxCores, executorMemory, command, sparkHome)
+ val sparkHome = sc.getSparkHome().getOrElse(
+ throw new IllegalArgumentException("must supply spark home for spark standalone"))
+ val appDesc = new ApplicationDescription(appName, maxCores, executorMemory, command, sparkHome)
- client = new Client(sc.env.actorSystem, master, jobDesc, this)
+ client = new Client(sc.env.actorSystem, master, appDesc, this)
client.start()
}
@@ -45,8 +46,8 @@ private[spark] class SparkDeploySchedulerBackend(
}
}
- override def connected(jobId: String) {
- logInfo("Connected to Spark cluster with job ID " + jobId)
+ override def connected(appId: String) {
+ logInfo("Connected to Spark cluster with app ID " + appId)
}
override def disconnected() {
diff --git a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
index 7caf06e917..f4a2994b6d 100644
--- a/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/CoarseMesosSchedulerBackend.scala
@@ -28,7 +28,7 @@ private[spark] class CoarseMesosSchedulerBackend(
scheduler: ClusterScheduler,
sc: SparkContext,
master: String,
- frameworkName: String)
+ appName: String)
extends StandaloneSchedulerBackend(scheduler, sc.env.actorSystem)
with MScheduler
with Logging {
@@ -76,7 +76,7 @@ private[spark] class CoarseMesosSchedulerBackend(
setDaemon(true)
override def run() {
val scheduler = CoarseMesosSchedulerBackend.this
- val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(frameworkName).build()
+ val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build()
driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
try { {
val ret = driver.run()
diff --git a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
index 300766d0f5..ca7fab4cc5 100644
--- a/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/spark/scheduler/mesos/MesosSchedulerBackend.scala
@@ -24,7 +24,7 @@ private[spark] class MesosSchedulerBackend(
scheduler: ClusterScheduler,
sc: SparkContext,
master: String,
- frameworkName: String)
+ appName: String)
extends SchedulerBackend
with MScheduler
with Logging {
@@ -49,7 +49,7 @@ private[spark] class MesosSchedulerBackend(
setDaemon(true)
override def run() {
val scheduler = MesosSchedulerBackend.this
- val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(frameworkName).build()
+ val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build()
driver = new MesosSchedulerDriver(scheduler, fwInfo, master)
try {
val ret = driver.run()
diff --git a/core/src/main/scala/spark/storage/BlockManager.scala b/core/src/main/scala/spark/storage/BlockManager.scala
index 2e7db60841..2462721fb8 100644
--- a/core/src/main/scala/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/spark/storage/BlockManager.scala
@@ -513,7 +513,7 @@ class BlockManager(
}
}
- // Split local and remote blocks. Remote blocks are further split into FetchRequests of size
+ // Partition 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) {
diff --git a/core/src/main/scala/spark/storage/StorageUtils.scala b/core/src/main/scala/spark/storage/StorageUtils.scala
index 5f72b67b2c..dec47a9d41 100644
--- a/core/src/main/scala/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/spark/storage/StorageUtils.scala
@@ -63,7 +63,7 @@ object StorageUtils {
val rddName = Option(rdd.name).getOrElse(rddKey)
val rddStorageLevel = rdd.getStorageLevel
- RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.splits.size, memSize, diskSize)
+ RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, rdd.partitions.size, memSize, diskSize)
}.toArray
}
diff --git a/core/src/main/twirl/spark/deploy/master/app_details.scala.html b/core/src/main/twirl/spark/deploy/master/app_details.scala.html
new file mode 100644
index 0000000000..301a7e2124
--- /dev/null
+++ b/core/src/main/twirl/spark/deploy/master/app_details.scala.html
@@ -0,0 +1,40 @@
+@(app: spark.deploy.master.ApplicationInfo)
+
+@spark.common.html.layout(title = "Application Details") {
+
+ <!-- Application Details -->
+ <div class="row">
+ <div class="span12">
+ <ul class="unstyled">
+ <li><strong>ID:</strong> @app.id</li>
+ <li><strong>Description:</strong> @app.desc.name</li>
+ <li><strong>User:</strong> @app.desc.user</li>
+ <li><strong>Cores:</strong>
+ @app.desc.cores
+ (@app.coresGranted Granted
+ @if(app.desc.cores == Integer.MAX_VALUE) {
+
+ } else {
+ , @app.coresLeft
+ }
+ )
+ </li>
+ <li><strong>Memory per Slave:</strong> @app.desc.memoryPerSlave</li>
+ <li><strong>Submit Date:</strong> @app.submitDate</li>
+ <li><strong>State:</strong> @app.state</li>
+ </ul>
+ </div>
+ </div>
+
+ <hr/>
+
+ <!-- Executors -->
+ <div class="row">
+ <div class="span12">
+ <h3> Executor Summary </h3>
+ <br/>
+ @executors_table(app.executors.values.toList)
+ </div>
+ </div>
+
+}
diff --git a/core/src/main/twirl/spark/deploy/master/app_row.scala.html b/core/src/main/twirl/spark/deploy/master/app_row.scala.html
new file mode 100644
index 0000000000..feb306f35c
--- /dev/null
+++ b/core/src/main/twirl/spark/deploy/master/app_row.scala.html
@@ -0,0 +1,20 @@
+@(app: spark.deploy.master.ApplicationInfo)
+
+@import spark.Utils
+@import spark.deploy.WebUI.formatDate
+@import spark.deploy.WebUI.formatDuration
+
+<tr>
+ <td>
+ <a href="app?appId=@(app.id)">@app.id</a>
+ </td>
+ <td>@app.desc.name</td>
+ <td>
+ @app.coresGranted
+ </td>
+ <td>@Utils.memoryMegabytesToString(app.desc.memoryPerSlave)</td>
+ <td>@formatDate(app.submitDate)</td>
+ <td>@app.desc.user</td>
+ <td>@app.state.toString()</td>
+ <td>@formatDuration(app.duration)</td>
+</tr>
diff --git a/core/src/main/twirl/spark/deploy/master/job_table.scala.html b/core/src/main/twirl/spark/deploy/master/app_table.scala.html
index d267d6e85e..f789cee0f1 100644
--- a/core/src/main/twirl/spark/deploy/master/job_table.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/app_table.scala.html
@@ -1,9 +1,9 @@
-@(jobs: Array[spark.deploy.master.JobInfo])
+@(apps: Array[spark.deploy.master.ApplicationInfo])
<table class="table table-bordered table-striped table-condensed sortable">
<thead>
<tr>
- <th>JobID</th>
+ <th>ID</th>
<th>Description</th>
<th>Cores</th>
<th>Memory per Node</th>
@@ -14,8 +14,8 @@
</tr>
</thead>
<tbody>
- @for(j <- jobs) {
- @job_row(j)
+ @for(j <- apps) {
+ @app_row(j)
}
</tbody>
</table>
diff --git a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
index 784d692fc2..d2d80fad48 100644
--- a/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/executor_row.scala.html
@@ -9,7 +9,7 @@
<td>@executor.memory</td>
<td>@executor.state</td>
<td>
- <a href="@(executor.worker.webUiAddress)/log?jobId=@(executor.job.id)&executorId=@(executor.id)&logType=stdout">stdout</a>
- <a href="@(executor.worker.webUiAddress)/log?jobId=@(executor.job.id)&executorId=@(executor.id)&logType=stderr">stderr</a>
+ <a href="@(executor.worker.webUiAddress)/log?appId=@(executor.application.id)&executorId=@(executor.id)&logType=stdout">stdout</a>
+ <a href="@(executor.worker.webUiAddress)/log?appId=@(executor.application.id)&executorId=@(executor.id)&logType=stderr">stderr</a>
</td>
-</tr> \ No newline at end of file
+</tr>
diff --git a/core/src/main/twirl/spark/deploy/master/index.scala.html b/core/src/main/twirl/spark/deploy/master/index.scala.html
index cb1651c7e1..ac51a39a51 100644
--- a/core/src/main/twirl/spark/deploy/master/index.scala.html
+++ b/core/src/main/twirl/spark/deploy/master/index.scala.html
@@ -14,7 +14,7 @@
@{state.workers.map(_.coresUsed).sum} Used</li>
<li><strong>Memory:</strong> @{Utils.memoryMegabytesToString(state.workers.map(_.memory).sum)} Total,
@{Utils.memoryMegabytesToString(state.workers.map(_.memoryUsed).sum)} Used</li>
- <li><strong>Jobs:</strong> @state.activeJobs.size Running, @state.completedJobs.size Completed </li>
+ <li><strong>Applications:</strong> @state.activeApps.size Running, @state.completedApps.size Completed </li>
</ul>
</div>
</div>
@@ -22,7 +22,7 @@
<!-- Worker Summary -->
<div class="row">
<div class="span12">
- <h3> Cluster Summary </h3>
+ <h3> Workers </h3>
<br/>
@worker_table(state.workers.sortBy(_.id))
</div>
@@ -30,23 +30,23 @@
<hr/>
- <!-- Job Summary (Running) -->
+ <!-- App Summary (Running) -->
<div class="row">
<div class="span12">
- <h3> Running Jobs </h3>
+ <h3> Running Applications </h3>
<br/>
- @job_table(state.activeJobs.sortBy(_.startTime).reverse)
+ @app_table(state.activeApps.sortBy(_.startTime).reverse)
</div>
</div>
<hr/>
- <!-- Job Summary (Completed) -->
+ <!-- App Summary (Completed) -->
<div class="row">
<div class="span12">
- <h3> Completed Jobs </h3>
+ <h3> Completed Applications </h3>
<br/>
- @job_table(state.completedJobs.sortBy(_.endTime).reverse)
+ @app_table(state.completedApps.sortBy(_.endTime).reverse)
</div>
</div>
diff --git a/core/src/main/twirl/spark/deploy/master/job_details.scala.html b/core/src/main/twirl/spark/deploy/master/job_details.scala.html
deleted file mode 100644
index d02a51b214..0000000000
--- a/core/src/main/twirl/spark/deploy/master/job_details.scala.html
+++ /dev/null
@@ -1,40 +0,0 @@
-@(job: spark.deploy.master.JobInfo)
-
-@spark.common.html.layout(title = "Job Details") {
-
- <!-- Job Details -->
- <div class="row">
- <div class="span12">
- <ul class="unstyled">
- <li><strong>ID:</strong> @job.id</li>
- <li><strong>Description:</strong> @job.desc.name</li>
- <li><strong>User:</strong> @job.desc.user</li>
- <li><strong>Cores:</strong>
- @job.desc.cores
- (@job.coresGranted Granted
- @if(job.desc.cores == Integer.MAX_VALUE) {
-
- } else {
- , @job.coresLeft
- }
- )
- </li>
- <li><strong>Memory per Slave:</strong> @job.desc.memoryPerSlave</li>
- <li><strong>Submit Date:</strong> @job.submitDate</li>
- <li><strong>State:</strong> @job.state</li>
- </ul>
- </div>
- </div>
-
- <hr/>
-
- <!-- Executors -->
- <div class="row">
- <div class="span12">
- <h3> Executor Summary </h3>
- <br/>
- @executors_table(job.executors.values.toList)
- </div>
- </div>
-
-}
diff --git a/core/src/main/twirl/spark/deploy/master/job_row.scala.html b/core/src/main/twirl/spark/deploy/master/job_row.scala.html
deleted file mode 100644
index 7c466a6a2c..0000000000
--- a/core/src/main/twirl/spark/deploy/master/job_row.scala.html
+++ /dev/null
@@ -1,20 +0,0 @@
-@(job: spark.deploy.master.JobInfo)
-
-@import spark.Utils
-@import spark.deploy.WebUI.formatDate
-@import spark.deploy.WebUI.formatDuration
-
-<tr>
- <td>
- <a href="job?jobId=@(job.id)">@job.id</a>
- </td>
- <td>@job.desc.name</td>
- <td>
- @job.coresGranted
- </td>
- <td>@Utils.memoryMegabytesToString(job.desc.memoryPerSlave)</td>
- <td>@formatDate(job.submitDate)</td>
- <td>@job.desc.user</td>
- <td>@job.state.toString()</td>
- <td>@formatDuration(job.duration)</td>
-</tr>
diff --git a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
index ea9542461e..dad0a89080 100644
--- a/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
+++ b/core/src/main/twirl/spark/deploy/worker/executor_row.scala.html
@@ -8,13 +8,13 @@
<td>@Utils.memoryMegabytesToString(executor.memory)</td>
<td>
<ul class="unstyled">
- <li><strong>ID:</strong> @executor.jobId</li>
- <li><strong>Name:</strong> @executor.jobDesc.name</li>
- <li><strong>User:</strong> @executor.jobDesc.user</li>
+ <li><strong>ID:</strong> @executor.appId</li>
+ <li><strong>Name:</strong> @executor.appDesc.name</li>
+ <li><strong>User:</strong> @executor.appDesc.user</li>
</ul>
</td>
<td>
- <a href="log?jobId=@(executor.jobId)&executorId=@(executor.execId)&logType=stdout">stdout</a>
- <a href="log?jobId=@(executor.jobId)&executorId=@(executor.execId)&logType=stderr">stderr</a>
+ <a href="log?appId=@(executor.appId)&executorId=@(executor.execId)&logType=stdout">stdout</a>
+ <a href="log?appId=@(executor.appId)&executorId=@(executor.execId)&logType=stderr">stderr</a>
</td>
</tr>
diff --git a/core/src/test/scala/spark/CheckpointSuite.scala b/core/src/test/scala/spark/CheckpointSuite.scala
index 0d08fd2396..3e5ffa81d6 100644
--- a/core/src/test/scala/spark/CheckpointSuite.scala
+++ b/core/src/test/scala/spark/CheckpointSuite.scala
@@ -34,7 +34,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
testCheckpointing(_.sample(false, 0.5, 0))
testCheckpointing(_.glom())
testCheckpointing(_.mapPartitions(_.map(_.toString)))
- testCheckpointing(r => new MapPartitionsWithSplitRDD(r,
+ testCheckpointing(r => new MapPartitionsWithIndexRDD(r,
(i: Int, iter: Iterator[Int]) => iter.map(_.toString), false ))
testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).mapValues(_.toString))
testCheckpointing(_.map(x => (x % 2, 1)).reduceByKey(_ + _).flatMapValues(x => 1 to x))
@@ -43,14 +43,14 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
test("ParallelCollection") {
val parCollection = sc.makeRDD(1 to 4, 2)
- val numSplits = parCollection.splits.size
+ val numPartitions = parCollection.partitions.size
parCollection.checkpoint()
assert(parCollection.dependencies === Nil)
val result = parCollection.collect()
assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result)
assert(parCollection.dependencies != Nil)
- assert(parCollection.splits.length === numSplits)
- assert(parCollection.splits.toList === parCollection.checkpointData.get.getSplits.toList)
+ assert(parCollection.partitions.length === numPartitions)
+ assert(parCollection.partitions.toList === parCollection.checkpointData.get.getPartitions.toList)
assert(parCollection.collect() === result)
}
@@ -59,13 +59,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
val blockManager = SparkEnv.get.blockManager
blockManager.putSingle(blockId, "test", StorageLevel.MEMORY_ONLY)
val blockRDD = new BlockRDD[String](sc, Array(blockId))
- val numSplits = blockRDD.splits.size
+ val numPartitions = blockRDD.partitions.size
blockRDD.checkpoint()
val result = blockRDD.collect()
assert(sc.checkpointFile[String](blockRDD.getCheckpointFile.get).collect() === result)
assert(blockRDD.dependencies != Nil)
- assert(blockRDD.splits.length === numSplits)
- assert(blockRDD.splits.toList === blockRDD.checkpointData.get.getSplits.toList)
+ assert(blockRDD.partitions.length === numPartitions)
+ assert(blockRDD.partitions.toList === blockRDD.checkpointData.get.getPartitions.toList)
assert(blockRDD.collect() === result)
}
@@ -79,9 +79,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
test("UnionRDD") {
def otherRDD = sc.makeRDD(1 to 10, 1)
- // Test whether the size of UnionRDDSplits reduce in size after parent RDD is checkpointed.
+ // Test whether the size of UnionRDDPartitions reduce in size after parent RDD is checkpointed.
// Current implementation of UnionRDD has transient reference to parent RDDs,
- // so only the splits will reduce in serialized size, not the RDD.
+ // so only the partitions will reduce in serialized size, not the RDD.
testCheckpointing(_.union(otherRDD), false, true)
testParentCheckpointing(_.union(otherRDD), false, true)
}
@@ -91,21 +91,21 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
testCheckpointing(new CartesianRDD(sc, _, otherRDD))
// Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
- // Current implementation of CoalescedRDDSplit has transient reference to parent RDD,
- // so only the RDD will reduce in serialized size, not the splits.
+ // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
+ // so only the RDD will reduce in serialized size, not the partitions.
testParentCheckpointing(new CartesianRDD(sc, _, otherRDD), true, false)
- // Test that the CartesianRDD updates parent splits (CartesianRDD.s1/s2) after
- // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits.
+ // Test that the CartesianRDD updates parent partitions (CartesianRDD.s1/s2) after
+ // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
// Note that this test is very specific to the current implementation of CartesianRDD.
val ones = sc.makeRDD(1 to 100, 10).map(x => x)
ones.checkpoint() // checkpoint that MappedRDD
val cartesian = new CartesianRDD(sc, ones, ones)
val splitBeforeCheckpoint =
- serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit])
+ serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition])
cartesian.count() // do the checkpointing
val splitAfterCheckpoint =
- serializeDeserialize(cartesian.splits.head.asInstanceOf[CartesianSplit])
+ serializeDeserialize(cartesian.partitions.head.asInstanceOf[CartesianPartition])
assert(
(splitAfterCheckpoint.s1 != splitBeforeCheckpoint.s1) &&
(splitAfterCheckpoint.s2 != splitBeforeCheckpoint.s2),
@@ -117,24 +117,24 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
testCheckpointing(_.coalesce(2))
// Test whether size of CoalescedRDD reduce in size after parent RDD is checkpointed
- // Current implementation of CoalescedRDDSplit has transient reference to parent RDD,
- // so only the RDD will reduce in serialized size, not the splits.
+ // Current implementation of CoalescedRDDPartition has transient reference to parent RDD,
+ // so only the RDD will reduce in serialized size, not the partitions.
testParentCheckpointing(_.coalesce(2), true, false)
- // Test that the CoalescedRDDSplit updates parent splits (CoalescedRDDSplit.parents) after
- // the parent RDD has been checkpointed and parent splits have been changed to HadoopSplits.
- // Note that this test is very specific to the current implementation of CoalescedRDDSplits
+ // Test that the CoalescedRDDPartition updates parent partitions (CoalescedRDDPartition.parents) after
+ // the parent RDD has been checkpointed and parent partitions have been changed to HadoopPartitions.
+ // Note that this test is very specific to the current implementation of CoalescedRDDPartitions
val ones = sc.makeRDD(1 to 100, 10).map(x => x)
ones.checkpoint() // checkpoint that MappedRDD
val coalesced = new CoalescedRDD(ones, 2)
val splitBeforeCheckpoint =
- serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit])
+ serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition])
coalesced.count() // do the checkpointing
val splitAfterCheckpoint =
- serializeDeserialize(coalesced.splits.head.asInstanceOf[CoalescedRDDSplit])
+ serializeDeserialize(coalesced.partitions.head.asInstanceOf[CoalescedRDDPartition])
assert(
splitAfterCheckpoint.parents.head != splitBeforeCheckpoint.parents.head,
- "CoalescedRDDSplit.parents not updated after parent RDD checkpointed"
+ "CoalescedRDDPartition.parents not updated after parent RDD checkpointed"
)
}
@@ -156,8 +156,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
// Test whether size of ZippedRDD reduce in size after parent RDD is checkpointed
- // Current implementation of ZippedRDDSplit has transient references to parent RDDs,
- // so only the RDD will reduce in serialized size, not the splits.
+ // Current implementation of ZippedRDDPartitions has transient references to parent RDDs,
+ // so only the RDD will reduce in serialized size, not the partitions.
testParentCheckpointing(
rdd => new ZippedRDD(sc, rdd, rdd.map(x => x)), true, false)
}
@@ -165,21 +165,21 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
/**
* Test checkpointing of the final RDD generated by the given operation. By default,
* this method tests whether the size of serialized RDD has reduced after checkpointing or not.
- * It can also test whether the size of serialized RDD splits has reduced after checkpointing or
- * not, but this is not done by default as usually the splits do not refer to any RDD and
+ * It can also test whether the size of serialized RDD partitions has reduced after checkpointing or
+ * not, but this is not done by default as usually the partitions do not refer to any RDD and
* therefore never store the lineage.
*/
def testCheckpointing[U: ClassManifest](
op: (RDD[Int]) => RDD[U],
testRDDSize: Boolean = true,
- testRDDSplitSize: Boolean = false
+ testRDDPartitionSize: Boolean = false
) {
// Generate the final RDD using given RDD operation
val baseRDD = generateLongLineageRDD()
val operatedRDD = op(baseRDD)
val parentRDD = operatedRDD.dependencies.headOption.orNull
val rddType = operatedRDD.getClass.getSimpleName
- val numSplits = operatedRDD.splits.length
+ val numPartitions = operatedRDD.partitions.length
// Find serialized sizes before and after the checkpoint
val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
@@ -193,11 +193,11 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
// Test whether dependencies have been changed from its earlier parent RDD
assert(operatedRDD.dependencies.head.rdd != parentRDD)
- // Test whether the splits have been changed to the new Hadoop splits
- assert(operatedRDD.splits.toList === operatedRDD.checkpointData.get.getSplits.toList)
+ // Test whether the partitions have been changed to the new Hadoop partitions
+ assert(operatedRDD.partitions.toList === operatedRDD.checkpointData.get.getPartitions.toList)
- // Test whether the number of splits is same as before
- assert(operatedRDD.splits.length === numSplits)
+ // Test whether the number of partitions is same as before
+ assert(operatedRDD.partitions.length === numPartitions)
// Test whether the data in the checkpointed RDD is same as original
assert(operatedRDD.collect() === result)
@@ -215,18 +215,18 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
)
}
- // Test whether serialized size of the splits has reduced. If the splits
- // do not have any non-transient reference to another RDD or another RDD's splits, it
+ // Test whether serialized size of the partitions has reduced. If the partitions
+ // do not have any non-transient reference to another RDD or another RDD's partitions, it
// does not refer to a lineage and therefore may not reduce in size after checkpointing.
- // However, if the original splits before checkpointing do refer to a parent RDD, the splits
+ // However, if the original partitions before checkpointing do refer to a parent RDD, the partitions
// must be forgotten after checkpointing (to remove all reference to parent RDDs) and
- // replaced with the HadoopSplits of the checkpointed RDD.
- if (testRDDSplitSize) {
- logInfo("Size of " + rddType + " splits "
+ // replaced with the HadooPartitions of the checkpointed RDD.
+ if (testRDDPartitionSize) {
+ logInfo("Size of " + rddType + " partitions "
+ "[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]")
assert(
splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
- "Size of " + rddType + " splits did not reduce after checkpointing " +
+ "Size of " + rddType + " partitions did not reduce after checkpointing " +
"[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
)
}
@@ -235,13 +235,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
/**
* Test whether checkpointing of the parent of the generated RDD also
* truncates the lineage or not. Some RDDs like CoGroupedRDD hold on to its parent
- * RDDs splits. So even if the parent RDD is checkpointed and its splits changed,
- * this RDD will remember the splits and therefore potentially the whole lineage.
+ * RDDs partitions. So even if the parent RDD is checkpointed and its partitions changed,
+ * this RDD will remember the partitions and therefore potentially the whole lineage.
*/
def testParentCheckpointing[U: ClassManifest](
op: (RDD[Int]) => RDD[U],
testRDDSize: Boolean,
- testRDDSplitSize: Boolean
+ testRDDPartitionSize: Boolean
) {
// Generate the final RDD using given RDD operation
val baseRDD = generateLongLineageRDD()
@@ -250,9 +250,9 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
val rddType = operatedRDD.getClass.getSimpleName
val parentRDDType = parentRDD.getClass.getSimpleName
- // Get the splits and dependencies of the parent in case they're lazily computed
+ // Get the partitions and dependencies of the parent in case they're lazily computed
parentRDD.dependencies
- parentRDD.splits
+ parentRDD.partitions
// Find serialized sizes before and after the checkpoint
val (rddSizeBeforeCheckpoint, splitSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD)
@@ -275,16 +275,16 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
)
}
- // Test whether serialized size of the splits has reduced because of its parent being
- // checkpointed. If the splits do not have any non-transient reference to another RDD
- // or another RDD's splits, it does not refer to a lineage and therefore may not reduce
- // in size after checkpointing. However, if the splits do refer to the *splits* of a parent
- // RDD, then these splits must update reference to the parent RDD splits as the parent RDD's
- // splits must have changed after checkpointing.
- if (testRDDSplitSize) {
+ // Test whether serialized size of the partitions has reduced because of its parent being
+ // checkpointed. If the partitions do not have any non-transient reference to another RDD
+ // or another RDD's partitions, it does not refer to a lineage and therefore may not reduce
+ // in size after checkpointing. However, if the partitions do refer to the *partitions* of a parent
+ // RDD, then these partitions must update reference to the parent RDD partitions as the parent RDD's
+ // partitions must have changed after checkpointing.
+ if (testRDDPartitionSize) {
assert(
splitSizeAfterCheckpoint < splitSizeBeforeCheckpoint,
- "Size of " + rddType + " splits did not reduce after checkpointing parent " + parentRDDType +
+ "Size of " + rddType + " partitions did not reduce after checkpointing parent " + parentRDDType +
"[" + splitSizeBeforeCheckpoint + " --> " + splitSizeAfterCheckpoint + "]"
)
}
@@ -321,12 +321,12 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
}
/**
- * Get serialized sizes of the RDD and its splits, in order to test whether the size shrinks
+ * Get serialized sizes of the RDD and its partitions, in order to test whether the size shrinks
* upon checkpointing. Ignores the checkpointData field, which may grow when we checkpoint.
*/
def getSerializedSizes(rdd: RDD[_]): (Int, Int) = {
(Utils.serialize(rdd).length - Utils.serialize(rdd.checkpointData).length,
- Utils.serialize(rdd.splits).length)
+ Utils.serialize(rdd.partitions).length)
}
/**
@@ -347,7 +347,7 @@ object CheckpointSuite {
def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = {
//println("First = " + first + ", second = " + second)
new CoGroupedRDD[K](
- Seq(first.asInstanceOf[RDD[(_, _)]], second.asInstanceOf[RDD[(_, _)]]),
+ Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]),
part
).asInstanceOf[RDD[(K, Seq[Seq[V]])]]
}
diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala
index ffa866de75..9739ba869b 100644
--- a/core/src/test/scala/spark/RDDSuite.scala
+++ b/core/src/test/scala/spark/RDDSuite.scala
@@ -33,6 +33,11 @@ class RDDSuite extends FunSuite with LocalSparkContext {
}
assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7)))
+ val partitionSumsWithIndex = nums.mapPartitionsWithIndex {
+ case(split, iter) => Iterator((split, iter.reduceLeft(_ + _)))
+ }
+ assert(partitionSumsWithIndex.collect().toList === List((0, 3), (1, 7)))
+
intercept[UnsupportedOperationException] {
nums.filter(_ > 5).reduce(_ + _)
}
@@ -97,12 +102,12 @@ class RDDSuite extends FunSuite with LocalSparkContext {
test("caching with failures") {
sc = new SparkContext("local", "test")
- val onlySplit = new Split { override def index: Int = 0 }
+ val onlySplit = new Partition { override def index: Int = 0 }
var shouldFail = true
val rdd = new RDD[Int](sc, Nil) {
- override def getSplits: Array[Split] = Array(onlySplit)
+ override def getPartitions: Array[Partition] = Array(onlySplit)
override val getDependencies = List[Dependency[_]]()
- override def compute(split: Split, context: TaskContext): Iterator[Int] = {
+ override def compute(split: Partition, context: TaskContext): Iterator[Int] = {
if (shouldFail) {
throw new Exception("injected failure")
} else {
@@ -168,7 +173,7 @@ class RDDSuite extends FunSuite with LocalSparkContext {
val data = sc.parallelize(1 to 10, 10)
// Note that split number starts from 0, so > 8 means only 10th partition left.
val prunedRdd = new PartitionPruningRDD(data, splitNum => splitNum > 8)
- assert(prunedRdd.splits.size === 1)
+ assert(prunedRdd.partitions.size === 1)
val prunedData = prunedRdd.collect()
assert(prunedData.size === 1)
assert(prunedData(0) === 10)
diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala
index 3493b9511f..92c3f67416 100644
--- a/core/src/test/scala/spark/ShuffleSuite.scala
+++ b/core/src/test/scala/spark/ShuffleSuite.scala
@@ -1,6 +1,7 @@
package spark
import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.HashSet
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
@@ -98,6 +99,28 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
val sums = pairs.reduceByKey(_+_, 10).collect()
assert(sums.toSet === Set((1, 7), (2, 1)))
}
+
+ test("reduceByKey with partitioner") {
+ sc = new SparkContext("local", "test")
+ val p = new Partitioner() {
+ def numPartitions = 2
+ def getPartition(key: Any) = key.asInstanceOf[Int]
+ }
+ val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p)
+ val sums = pairs.reduceByKey(_+_)
+ assert(sums.collect().toSet === Set((1, 4), (0, 1)))
+ assert(sums.partitioner === Some(p))
+ // count the dependencies to make sure there is only 1 ShuffledRDD
+ val deps = new HashSet[RDD[_]]()
+ def visit(r: RDD[_]) {
+ for (dep <- r.dependencies) {
+ deps += dep.rdd
+ visit(dep.rdd)
+ }
+ }
+ visit(sums)
+ assert(deps.size === 2) // ShuffledRDD, ParallelCollection
+ }
test("join") {
sc = new SparkContext("local", "test")
@@ -199,7 +222,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext {
sc = new SparkContext("local", "test")
val emptyDir = Files.createTempDir()
val file = sc.textFile(emptyDir.getAbsolutePath)
- assert(file.splits.size == 0)
+ assert(file.partitions.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)
diff --git a/core/src/test/scala/spark/SortingSuite.scala b/core/src/test/scala/spark/SortingSuite.scala
index edb8c839fc..495f957e53 100644
--- a/core/src/test/scala/spark/SortingSuite.scala
+++ b/core/src/test/scala/spark/SortingSuite.scala
@@ -19,7 +19,7 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w
val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
val pairs = sc.parallelize(pairArr, 2)
val sorted = pairs.sortByKey()
- assert(sorted.splits.size === 2)
+ assert(sorted.partitions.size === 2)
assert(sorted.collect() === pairArr.sortBy(_._1))
}
@@ -29,17 +29,17 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w
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.partitions.size === 1)
assert(sorted.collect() === pairArr.sortBy(_._1))
}
- test("large array with many splits") {
+ test("large array with many partitions") {
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.partitions.size === 20)
assert(sorted.collect() === pairArr.sortBy(_._1))
}
@@ -59,7 +59,7 @@ class SortingSuite extends FunSuite with LocalSparkContext with ShouldMatchers w
assert(pairs.sortByKey(false, 1).collect() === pairArr.sortWith((x, y) => x._1 > y._1))
}
- test("sort descending with many splits") {
+ test("sort descending with many partitions") {
sc = new SparkContext("local", "test")
val rand = new scala.util.Random()
val pairArr = Array.fill(1000) { (rand.nextInt(), rand.nextInt()) }
diff --git a/core/src/test/scala/spark/ParallelCollectionSplitSuite.scala b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
index 450c69bd58..d27a2538e4 100644
--- a/core/src/test/scala/spark/ParallelCollectionSplitSuite.scala
+++ b/core/src/test/scala/spark/rdd/ParallelCollectionSplitSuite.scala
@@ -1,4 +1,4 @@
-package spark
+package spark.rdd
import scala.collection.immutable.NumericRange
@@ -11,7 +11,7 @@ import org.scalacheck.Prop._
class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("one element per slice") {
val data = Array(1, 2, 3)
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices(0).mkString(",") === "1")
assert(slices(1).mkString(",") === "2")
@@ -20,14 +20,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("one slice") {
val data = Array(1, 2, 3)
- val slices = ParallelCollection.slice(data, 1)
+ val slices = ParallelCollectionRDD.slice(data, 1)
assert(slices.size === 1)
assert(slices(0).mkString(",") === "1,2,3")
}
test("equal slices") {
val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9)
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices(0).mkString(",") === "1,2,3")
assert(slices(1).mkString(",") === "4,5,6")
@@ -36,7 +36,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("non-equal slices") {
val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices(0).mkString(",") === "1,2,3")
assert(slices(1).mkString(",") === "4,5,6")
@@ -45,7 +45,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("splitting exclusive range") {
val data = 0 until 100
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices(0).mkString(",") === (0 to 32).mkString(","))
assert(slices(1).mkString(",") === (33 to 65).mkString(","))
@@ -54,7 +54,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("splitting inclusive range") {
val data = 0 to 100
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices(0).mkString(",") === (0 to 32).mkString(","))
assert(slices(1).mkString(",") === (33 to 66).mkString(","))
@@ -63,24 +63,24 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("empty data") {
val data = new Array[Int](0)
- val slices = ParallelCollection.slice(data, 5)
+ val slices = ParallelCollectionRDD.slice(data, 5)
assert(slices.size === 5)
for (slice <- slices) assert(slice.size === 0)
}
test("zero slices") {
val data = Array(1, 2, 3)
- intercept[IllegalArgumentException] { ParallelCollection.slice(data, 0) }
+ intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, 0) }
}
test("negative number of slices") {
val data = Array(1, 2, 3)
- intercept[IllegalArgumentException] { ParallelCollection.slice(data, -5) }
+ intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, -5) }
}
test("exclusive ranges sliced into ranges") {
val data = 1 until 100
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices.map(_.size).reduceLeft(_+_) === 99)
assert(slices.forall(_.isInstanceOf[Range]))
@@ -88,7 +88,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("inclusive ranges sliced into ranges") {
val data = 1 to 100
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices.map(_.size).reduceLeft(_+_) === 100)
assert(slices.forall(_.isInstanceOf[Range]))
@@ -97,7 +97,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("large ranges don't overflow") {
val N = 100 * 1000 * 1000
val data = 0 until N
- val slices = ParallelCollection.slice(data, 40)
+ val slices = ParallelCollectionRDD.slice(data, 40)
assert(slices.size === 40)
for (i <- 0 until 40) {
assert(slices(i).isInstanceOf[Range])
@@ -117,7 +117,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
(tuple: (List[Int], Int)) =>
val d = tuple._1
val n = tuple._2
- val slices = ParallelCollection.slice(d, n)
+ val slices = ParallelCollectionRDD.slice(d, n)
("n slices" |: slices.size == n) &&
("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) &&
("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1))
@@ -134,7 +134,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
} yield (a until b by step, n)
val prop = forAll(gen) {
case (d: Range, n: Int) =>
- val slices = ParallelCollection.slice(d, n)
+ val slices = ParallelCollectionRDD.slice(d, n)
("n slices" |: slices.size == n) &&
("all ranges" |: slices.forall(_.isInstanceOf[Range])) &&
("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) &&
@@ -152,7 +152,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
} yield (a to b by step, n)
val prop = forAll(gen) {
case (d: Range, n: Int) =>
- val slices = ParallelCollection.slice(d, n)
+ val slices = ParallelCollectionRDD.slice(d, n)
("n slices" |: slices.size == n) &&
("all ranges" |: slices.forall(_.isInstanceOf[Range])) &&
("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) &&
@@ -163,7 +163,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("exclusive ranges of longs") {
val data = 1L until 100L
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices.map(_.size).reduceLeft(_+_) === 99)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
@@ -171,7 +171,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("inclusive ranges of longs") {
val data = 1L to 100L
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices.map(_.size).reduceLeft(_+_) === 100)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
@@ -179,7 +179,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("exclusive ranges of doubles") {
val data = 1.0 until 100.0 by 1.0
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices.map(_.size).reduceLeft(_+_) === 99)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
@@ -187,7 +187,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers {
test("inclusive ranges of doubles") {
val data = 1.0 to 100.0 by 1.0
- val slices = ParallelCollection.slice(data, 3)
+ val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
assert(slices.map(_.size).reduceLeft(_+_) === 100)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
diff --git a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
index 83663ac702..8de490eb86 100644
--- a/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
+++ b/core/src/test/scala/spark/scheduler/DAGSchedulerSuite.scala
@@ -24,7 +24,7 @@ import spark.MapOutputTracker
import spark.RDD
import spark.SparkContext
import spark.SparkException
-import spark.Split
+import spark.Partition
import spark.TaskContext
import spark.TaskEndReason
@@ -144,18 +144,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
* so we can test that DAGScheduler does not try to execute RDDs locally.
*/
def makeRdd(
- numSplits: Int,
+ numPartitions: Int,
dependencies: List[Dependency[_]],
locations: Seq[Seq[String]] = Nil
): MyRDD = {
- val maxSplit = numSplits - 1
+ val maxPartition = numPartitions - 1
return new MyRDD(sc, dependencies) {
- override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] =
+ override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] =
throw new RuntimeException("should not be reached")
- override def getSplits() = (0 to maxSplit).map(i => new Split {
+ override def getPartitions = (0 to maxPartition).map(i => new Partition {
override def index = i
}).toArray
- override def getPreferredLocations(split: Split): Seq[String] =
+ override def getPreferredLocations(split: Partition): Seq[String] =
if (locations.isDefinedAt(split.index))
locations(split.index)
else
@@ -295,11 +295,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
* collect the result of the job via callbacks from DAGScheduler.
*/
def submitRdd(rdd: MyRDD, allowLocal: Boolean = false): (JobWaiter[Int], Array[Int]) = {
- val resultArray = new Array[Int](rdd.splits.size)
+ val resultArray = new Array[Int](rdd.partitions.size)
val (toSubmit, waiter) = scheduler.prepareJob[(Int, Int), Int](
rdd,
jobComputeFunc,
- (0 to (rdd.splits.size - 1)),
+ (0 to (rdd.partitions.size - 1)),
"test-site",
allowLocal,
(i: Int, value: Int) => resultArray(i) = value
@@ -355,10 +355,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar
test("local job") {
val rdd = new MyRDD(sc, Nil) {
- override def compute(split: Split, context: TaskContext): Iterator[(Int, Int)] =
+ override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] =
Array(42 -> 0).iterator
- override def getSplits() = Array( new Split { override def index = 0 } )
- override def getPreferredLocations(split: Split) = Nil
+ override def getPartitions = Array( new Partition { override def index = 0 } )
+ override def getPreferredLocations(split: Partition) = Nil
override def toString = "DAGSchedulerSuite Local RDD"
}
submitRdd(rdd, true)
diff --git a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
index a5db7103f5..647bcaf860 100644
--- a/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
+++ b/core/src/test/scala/spark/scheduler/TaskContextSuite.scala
@@ -5,7 +5,7 @@ import org.scalatest.BeforeAndAfter
import spark.TaskContext
import spark.RDD
import spark.SparkContext
-import spark.Split
+import spark.Partition
import spark.LocalSparkContext
class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext {
@@ -14,8 +14,8 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte
var completed = false
sc = new SparkContext("local", "test")
val rdd = new RDD[String](sc, List()) {
- override def getSplits = Array[Split](StubSplit(0))
- override def compute(split: Split, context: TaskContext) = {
+ override def getPartitions = Array[Partition](StubPartition(0))
+ override def compute(split: Partition, context: TaskContext) = {
context.addOnCompleteCallback(() => completed = true)
sys.error("failed")
}
@@ -28,5 +28,5 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte
assert(completed === true)
}
- case class StubSplit(val index: Int) extends Split
-} \ No newline at end of file
+ case class StubPartition(val index: Int) extends Partition
+}