From 3ce3a282c8463408f9a2db93c1748e8df8087e07 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Wed, 7 Sep 2016 10:04:00 +0100 Subject: [SPARK-17359][SQL][MLLIB] Use ArrayBuffer.+=(A) instead of ArrayBuffer.append(A) in performance critical paths ## What changes were proposed in this pull request? We should generally use `ArrayBuffer.+=(A)` rather than `ArrayBuffer.append(A)`, because `append(A)` would involve extra boxing / unboxing. ## How was this patch tested? N/A Author: Liwei Lin Closes #14914 from lw-lin/append_to_plus_eq_v2. --- .../spark/sql/catalyst/expressions/ExpressionSet.scala | 2 +- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 4 ++-- .../apache/spark/sql/catalyst/util/QuantileSummaries.scala | 8 ++++---- .../org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala | 12 ++++++------ .../org/apache/spark/sql/execution/DataSourceScanExec.scala | 4 ++-- .../scala/org/apache/spark/sql/execution/WindowExec.scala | 4 ++-- .../org/apache/spark/sql/execution/streaming/memory.scala | 4 ++-- 7 files changed, 19 insertions(+), 19 deletions(-) (limited to 'sql') diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala index 644a5b28a2..f93e5736de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -55,7 +55,7 @@ class ExpressionSet protected( protected def add(e: Expression): Unit = { if (!baseSet.contains(e.canonicalized)) { baseSet.add(e.canonicalized) - originals.append(e) + originals += e } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 28064a5cfa..f982c222af 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -596,12 +596,12 @@ class CodegenContext { // also not be too small, or it will have many function calls (for wide table), see the // results in BenchmarkWideTable. if (blockBuilder.length > 1024) { - blocks.append(blockBuilder.toString()) + blocks += blockBuilder.toString() blockBuilder.clear() } blockBuilder.append(code) } - blocks.append(blockBuilder.toString()) + blocks += blockBuilder.toString() if (blocks.length == 1) { // inline execution if only one block diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala index 493b5faf9e..7512ace188 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala @@ -59,7 +59,7 @@ class QuantileSummaries( * @param x the new observation to insert into the summary */ def insert(x: Double): QuantileSummaries = { - headSampled.append(x) + headSampled += x if (headSampled.size >= defaultHeadSize) { this.withHeadBufferInserted } else { @@ -90,7 +90,7 @@ class QuantileSummaries( val currentSample = sorted(opsIdx) // Add all the samples before the next observation. while(sampleIdx < sampled.size && sampled(sampleIdx).value <= currentSample) { - newSamples.append(sampled(sampleIdx)) + newSamples += sampled(sampleIdx) sampleIdx += 1 } @@ -104,13 +104,13 @@ class QuantileSummaries( } val tuple = Stats(currentSample, 1, delta) - newSamples.append(tuple) + newSamples += tuple opsIdx += 1 } // Add all the remaining existing samples while(sampleIdx < sampled.size) { - newSamples.append(sampled(sampleIdx)) + newSamples += sampled(sampleIdx) sampleIdx += 1 } new QuantileSummaries(compressThreshold, relativeError, newSamples.toArray, currentCount) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6a188e7e55..6246380dbe 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -82,8 +82,8 @@ class TreeNodeSuite extends SparkFunSuite { val expected = Seq("+", "1", "*", "2", "-", "3", "4") val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) expression transformDown { - case b: BinaryOperator => actual.append(b.symbol); b - case l: Literal => actual.append(l.toString); l + case b: BinaryOperator => actual += b.symbol; b + case l: Literal => actual += l.toString; l } assert(expected === actual) @@ -94,8 +94,8 @@ class TreeNodeSuite extends SparkFunSuite { val expected = Seq("1", "2", "3", "4", "-", "*", "+") val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) expression transformUp { - case b: BinaryOperator => actual.append(b.symbol); b - case l: Literal => actual.append(l.toString); l + case b: BinaryOperator => actual += b.symbol; b + case l: Literal => actual += l.toString; l } assert(expected === actual) @@ -134,8 +134,8 @@ class TreeNodeSuite extends SparkFunSuite { val expected = Seq("1", "2", "3", "4", "-", "*", "+") val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) expression foreachUp { - case b: BinaryOperator => actual.append(b.symbol); - case l: Literal => actual.append(l.toString); + case b: BinaryOperator => actual += b.symbol; + case l: Literal => actual += l.toString; } assert(expected === actual) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 1a8d0e310a..9597bdf34b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -448,7 +448,7 @@ case class FileSourceScanExec( FilePartition( partitions.size, currentFiles.toArray.toSeq) // Copy to a new Array. - partitions.append(newPartition) + partitions += newPartition } currentFiles.clear() currentSize = 0 @@ -462,7 +462,7 @@ case class FileSourceScanExec( } // Add the given file to the current partition. currentSize += file.length + openCostInBytes - currentFiles.append(file) + currentFiles += file } closePartition() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala index b60f17cc17..9d006d21d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala @@ -167,8 +167,8 @@ case class WindowExec( val key = (tpe, fr.frameType, FrameBoundary(fr.frameStart), FrameBoundary(fr.frameEnd)) val (es, fns) = framedFunctions.getOrElseUpdate( key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression])) - es.append(e) - fns.append(fn) + es += e + fns += fn } // Collect all valid window functions and group them by their frame. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index e37f0c7779..5052c4d50c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -77,7 +77,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) logDebug(s"Adding ds: $ds") this.synchronized { currentOffset = currentOffset + 1 - batches.append(ds) + batches += ds currentOffset } } @@ -155,7 +155,7 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi case InternalOutputModes.Complete => batches.clear() - batches.append(AddedData(batchId, data.collect())) + batches += AddedData(batchId, data.collect()) case _ => throw new IllegalArgumentException( -- cgit v1.2.3