diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 330569a8d8..f917cfd141 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -43,8 +43,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def rdd: RDD[T] - /** Set of partitions in this RDD. */ + @deprecated("Use partitions() instead.", "1.1.0") def splits: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) + + /** Set of partitions in this RDD. */ + def partitions: JList[Partition] = new java.util.ArrayList(rdd.partitions.toSeq) /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context: SparkContext = rdd.context diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 761f2d6a77..1d7a7be6cf 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -741,7 +741,7 @@ public class JavaAPISuite implements Serializable { public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics()); - Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); + Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } @Test diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 062bec2381..95c54e7a5a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -704,7 +704,7 @@ class SparkContext(object): [0, 1, 16, 25] """ if partitions == None: - partitions = range(rdd._jrdd.splits().size()) + partitions = range(rdd._jrdd.partitions().size()) javaPartitions = ListConverter().convert(partitions, self._gateway._gateway_client) # Implementation note: This is implemented as a mapPartitions followed diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1d55c35a8b..f64f48e3a4 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -321,7 +321,7 @@ class RDD(object): >>> rdd.getNumPartitions() 2 """ - return self._jrdd.splits().size() + return self._jrdd.partitions().size() def filter(self, f): """ @@ -922,7 +922,7 @@ class RDD(object): [91, 92, 93] """ items = [] - totalParts = self._jrdd.splits().size() + totalParts = self._jrdd.partitions().size() partsScanned = 0 while len(items) < num and partsScanned < totalParts: