Skip to content

Commit 77ec261

Browse files
committed
Replace use of .size with .length for Arrays.
Invoking .size on arrays is valid, but requires an implicit conversion to SeqLike. This incurs a compile time overhead and more importantly a runtime overhead, as the Array must be wrapped before the method can be invoked. For example, the difference in generated byte code is: public int withSize(); Code: 0: getstatic #23 // Field scala/Predef$.MODULE$:Lscala/Predef$; 3: aload_0 4: invokevirtual #25 // Method array:()[I 7: invokevirtual #29 // Method scala/Predef$.intArrayOps:([I)Lscala/collection/mutable/ArrayOps; 10: invokeinterface #34, 1 // InterfaceMethod scala/collection/mutable/ArrayOps.size:()I 15: ireturn public int withLength(); Code: 0: aload_0 1: invokevirtual #25 // Method array:()[I 4: arraylength 5: ireturn
1 parent 30363ed commit 77ec261

19 files changed

+42
-42
lines changed

core/src/main/scala/org/apache/spark/network/nio/Connection.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -181,7 +181,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector,
181181
buffer.get(bytes)
182182
bytes.foreach(x => print(x + " "))
183183
buffer.position(curPosition)
184-
print(" (" + bytes.size + ")")
184+
print(" (" + bytes.length + ")")
185185
}
186186

187187
def printBuffer(buffer: ByteBuffer, position: Int, length: Int) {

core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
4545
}
4646
result
4747
},
48-
Range(0, self.partitions.size),
48+
Range(0, self.partitions.length),
4949
(index: Int, data: Long) => totalCount.addAndGet(data),
5050
totalCount.get())
5151
}
@@ -54,8 +54,8 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
5454
* Returns a future for retrieving all elements of this RDD.
5555
*/
5656
def collectAsync(): FutureAction[Seq[T]] = {
57-
val results = new Array[Array[T]](self.partitions.size)
58-
self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size),
57+
val results = new Array[Array[T]](self.partitions.length)
58+
self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.length),
5959
(index, data) => results(index) = data, results.flatten.toSeq)
6060
}
6161

@@ -111,15 +111,15 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi
111111
*/
112112
def foreachAsync(f: T => Unit): FutureAction[Unit] = {
113113
val cleanF = self.context.clean(f)
114-
self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.size),
114+
self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.length),
115115
(index, data) => Unit, Unit)
116116
}
117117

118118
/**
119119
* Applies a function f to each partition of this RDD.
120120
*/
121121
def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = {
122-
self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size),
122+
self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.length),
123123
(index, data) => Unit, Unit)
124124
}
125125
}

core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds
3636

3737
override def getPartitions: Array[Partition] = {
3838
assertValid()
39-
(0 until blockIds.size).map(i => {
39+
(0 until blockIds.length).map(i => {
4040
new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition]
4141
}).toArray
4242
}

core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -53,11 +53,11 @@ class CartesianRDD[T: ClassTag, U: ClassTag](
5353
extends RDD[Pair[T, U]](sc, Nil)
5454
with Serializable {
5555

56-
val numPartitionsInRdd2 = rdd2.partitions.size
56+
val numPartitionsInRdd2 = rdd2.partitions.length
5757

5858
override def getPartitions: Array[Partition] = {
5959
// create the cross product split
60-
val array = new Array[Partition](rdd1.partitions.size * rdd2.partitions.size)
60+
val array = new Array[Partition](rdd1.partitions.length * rdd2.partitions.length)
6161
for (s1 <- rdd1.partitions; s2 <- rdd2.partitions) {
6262
val idx = s1.index * numPartitionsInRdd2 + s2.index
6363
array(idx) = new CartesianPartition(idx, rdd1, rdd2, s1.index, s2.index)

core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,7 @@ class CheckpointRDD[T: ClassTag](sc: SparkContext, val checkpointPath: String)
4949
if (fs.exists(cpath)) {
5050
val dirContents = fs.listStatus(cpath).map(_.getPath)
5151
val partitionFiles = dirContents.filter(_.getName.startsWith("part-")).map(_.toString).sorted
52-
val numPart = partitionFiles.size
52+
val numPart = partitionFiles.length
5353
if (numPart > 0 && (! partitionFiles(0).endsWith(CheckpointRDD.splitIdToFile(0)) ||
5454
! partitionFiles(numPart-1).endsWith(CheckpointRDD.splitIdToFile(numPart-1)))) {
5555
throw new SparkException("Invalid checkpoint directory: " + checkpointPath)

core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
9999

100100
override def getPartitions: Array[Partition] = {
101101
val array = new Array[Partition](part.numPartitions)
102-
for (i <- 0 until array.size) {
102+
for (i <- 0 until array.length) {
103103
// Each CoGroupPartition will have a dependency per contributing RDD
104104
array(i) = new CoGroupPartition(i, rdds.zipWithIndex.map { case (rdd, j) =>
105105
// Assume each RDD contributed a single dependency, and get it
@@ -120,7 +120,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
120120
val sparkConf = SparkEnv.get.conf
121121
val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true)
122122
val split = s.asInstanceOf[CoGroupPartition]
123-
val numRdds = split.deps.size
123+
val numRdds = split.deps.length
124124

125125
// A list of (rdd iterator, dependency number) pairs
126126
val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)]

core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -166,7 +166,7 @@ private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack:
166166

167167
// determines the tradeoff between load-balancing the partitions sizes and their locality
168168
// e.g. balanceSlack=0.10 means that it allows up to 10% imbalance in favor of locality
169-
val slack = (balanceSlack * prev.partitions.size).toInt
169+
val slack = (balanceSlack * prev.partitions.length).toInt
170170

171171
var noLocality = true // if true if no preferredLocations exists for parent RDD
172172

core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
7070
@Experimental
7171
def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
7272
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
73-
val evaluator = new MeanEvaluator(self.partitions.size, confidence)
73+
val evaluator = new MeanEvaluator(self.partitions.length, confidence)
7474
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
7575
}
7676

@@ -81,7 +81,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable {
8181
@Experimental
8282
def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = {
8383
val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns)
84-
val evaluator = new SumEvaluator(self.partitions.size, confidence)
84+
val evaluator = new SumEvaluator(self.partitions.length, confidence)
8585
self.context.runApproximateJob(self, processPartition, evaluator, timeout)
8686
}
8787

core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -56,7 +56,7 @@ class OrderedRDDFunctions[K : Ordering : ClassTag,
5656
* order of the keys).
5757
*/
5858
// TODO: this currently doesn't work on P other than Tuple2!
59-
def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size)
59+
def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.length)
6060
: RDD[(K, V)] =
6161
{
6262
val part = new RangePartitioner(numPartitions, self, ascending)

core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -823,7 +823,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
823823
* RDD will be <= us.
824824
*/
825825
def subtractByKey[W: ClassTag](other: RDD[(K, W)]): RDD[(K, V)] =
826-
subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.size)))
826+
subtractByKey(other, self.partitioner.getOrElse(new HashPartitioner(self.partitions.length)))
827827

828828
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
829829
def subtractByKey[W: ClassTag](other: RDD[(K, W)], numPartitions: Int): RDD[(K, V)] =

0 commit comments

Comments
 (0)