aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/WebUI.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala6
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala3
-rw-r--r--core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala4
-rw-r--r--external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala22
-rw-r--r--mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala12
-rw-r--r--mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala8
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala12
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/WindowExec.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala2
-rw-r--r--yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala4
25 files changed, 60 insertions, 61 deletions
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index 2822eb5d60..d841091a31 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -467,7 +467,7 @@ private[spark] object PythonRDD extends Logging {
val length = file.readInt()
val obj = new Array[Byte](length)
file.readFully(obj)
- objs.append(obj)
+ objs += obj
}
} catch {
case eof: EOFException => // No-op
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index 2c40e72699..38363800ec 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -83,8 +83,8 @@ private[spark] abstract class WebUI(
(request: HttpServletRequest) => page.renderJson(request), securityManager, conf, basePath)
attachHandler(renderHandler)
attachHandler(renderJsonHandler)
- pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
- .append(renderHandler)
+ val handlers = pageToHandlers.getOrElseUpdate(page, ArrayBuffer[ServletContextHandler]())
+ handlers += renderHandler
}
/** Attach a handler to this UI. */
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 0943528119..948cc3b099 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -184,7 +184,7 @@ class ExternalAppendOnlyMap[K, V, C](
override protected[this] def spill(collection: SizeTracker): Unit = {
val inMemoryIterator = currentMap.destructiveSortedIterator(keyComparator)
val diskMapIterator = spillMemoryIteratorToDisk(inMemoryIterator)
- spilledMaps.append(diskMapIterator)
+ spilledMaps += diskMapIterator
}
/**
@@ -215,7 +215,7 @@ class ExternalAppendOnlyMap[K, V, C](
// Flush the disk writer's contents to disk, and update relevant variables
def flush(): Unit = {
val segment = writer.commitAndGet()
- batchSizes.append(segment.length)
+ batchSizes += segment.length
_diskBytesSpilled += segment.length
objectsWritten = 0
}
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index 3579918fac..176f84fa2a 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -238,7 +238,7 @@ private[spark] class ExternalSorter[K, V, C](
override protected[this] def spill(collection: WritablePartitionedPairCollection[K, C]): Unit = {
val inMemoryIterator = collection.destructiveSortedWritablePartitionedIterator(comparator)
val spillFile = spillMemoryIteratorToDisk(inMemoryIterator)
- spills.append(spillFile)
+ spills += spillFile
}
/**
@@ -285,7 +285,7 @@ private[spark] class ExternalSorter[K, V, C](
// The writer is committed at the end of this process.
def flush(): Unit = {
val segment = writer.commitAndGet()
- batchSizes.append(segment.length)
+ batchSizes += segment.length
_diskBytesSpilled += segment.length
objectsWritten = 0
}
@@ -796,7 +796,7 @@ private[spark] class ExternalSorter[K, V, C](
logInfo(s"Task ${context.taskAttemptId} force spilling in-memory map to disk and " +
s" it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory")
val spillFile = spillMemoryIteratorToDisk(inMemoryIterator)
- forceSpillFiles.append(spillFile)
+ forceSpillFiles += spillFile
val spillReader = new SpillReader(spillFile)
nextUpstream = (0 until numPartitions).iterator.flatMap { p =>
val iterator = spillReader.readNextPartition()
diff --git a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
index 9ecf49b598..c9b3d657c2 100644
--- a/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/IvyTestUtils.scala
@@ -305,7 +305,7 @@ private[deploy] object IvyTestUtils {
val allFiles = ArrayBuffer[(String, File)](javaFile)
if (withPython) {
val pythonFile = createPythonFile(root)
- allFiles.append((pythonFile.getName, pythonFile))
+ allFiles += Tuple2(pythonFile.getName, pythonFile)
}
if (withR) {
val rFiles = createRFiles(root, className, artifact.groupId)
diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
index 38bf7e5e5a..eb2b3ffd15 100644
--- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala
@@ -118,8 +118,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft
if (numBytesToFree <= mm.storageMemoryUsed) {
// We can evict enough blocks to fulfill the request for space
mm.releaseStorageMemory(numBytesToFree, MemoryMode.ON_HEAP)
- evictedBlocks.append(
- (null, BlockStatus(StorageLevel.MEMORY_ONLY, numBytesToFree, 0L)))
+ evictedBlocks += Tuple2(null, BlockStatus(StorageLevel.MEMORY_ONLY, numBytesToFree, 0L))
numBytesToFree
} else {
// No blocks were evicted because eviction would not free enough space.
diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala
index ed9428820f..442941685f 100644
--- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala
@@ -107,7 +107,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte
val blockId = new TempShuffleBlockId(UUID.randomUUID)
val file = new File(tempDir, blockId.name)
blockIdToFileMap.put(blockId, file)
- temporaryFilesCreated.append(file)
+ temporaryFilesCreated += file
(blockId, file)
}
})
diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
index c342b68f46..2695295d45 100644
--- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala
@@ -150,12 +150,12 @@ class SizeEstimatorSuite
val buf = new ArrayBuffer[DummyString]()
for (i <- 0 until 5000) {
- buf.append(new DummyString(new Array[Char](10)))
+ buf += new DummyString(new Array[Char](10))
}
assertResult(340016)(SizeEstimator.estimate(buf.toArray))
for (i <- 0 until 5000) {
- buf.append(new DummyString(arr))
+ buf += new DummyString(arr)
}
assertResult(683912)(SizeEstimator.estimate(buf.toArray))
diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
index 726b5d8ec3..35acb7b09f 100644
--- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
+++ b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala
@@ -108,7 +108,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
} else {
val missing = topicAndPartitions.diff(leaderMap.keySet)
val err = new Err
- err.append(new SparkException(s"Couldn't find leaders for ${missing}"))
+ err += new SparkException(s"Couldn't find leaders for ${missing}")
Left(err)
}
}
@@ -139,7 +139,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
respErrs.foreach { m =>
val cause = ErrorMapping.exceptionFor(m.errorCode)
val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?"
- errs.append(new SparkException(msg, cause))
+ errs += new SparkException(msg, cause)
}
}
}
@@ -205,11 +205,11 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
LeaderOffset(consumer.host, consumer.port, off)
}
} else {
- errs.append(new SparkException(
- s"Empty offsets for ${tp}, is ${before} before log beginning?"))
+ errs += new SparkException(
+ s"Empty offsets for ${tp}, is ${before} before log beginning?")
}
} else {
- errs.append(ErrorMapping.exceptionFor(por.error))
+ errs += ErrorMapping.exceptionFor(por.error)
}
}
}
@@ -218,7 +218,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
}
}
val missing = topicAndPartitions.diff(result.keySet)
- errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}"))
+ errs += new SparkException(s"Couldn't find leader offsets for ${missing}")
Left(errs)
}
}
@@ -274,7 +274,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
if (ome.error == ErrorMapping.NoError) {
result += tp -> ome
} else {
- errs.append(ErrorMapping.exceptionFor(ome.error))
+ errs += ErrorMapping.exceptionFor(ome.error)
}
}
}
@@ -283,7 +283,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
}
}
val missing = topicAndPartitions.diff(result.keySet)
- errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}"))
+ errs += new SparkException(s"Couldn't find consumer offsets for ${missing}")
Left(errs)
}
@@ -330,7 +330,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
if (err == ErrorMapping.NoError) {
result += tp -> err
} else {
- errs.append(ErrorMapping.exceptionFor(err))
+ errs += ErrorMapping.exceptionFor(err)
}
}
}
@@ -339,7 +339,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
}
}
val missing = topicAndPartitions.diff(result.keySet)
- errs.append(new SparkException(s"Couldn't set offsets for ${missing}"))
+ errs += new SparkException(s"Couldn't set offsets for ${missing}")
Left(errs)
}
@@ -353,7 +353,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable {
fn(consumer)
} catch {
case NonFatal(e) =>
- errs.append(e)
+ errs += e
} finally {
if (consumer != null) {
consumer.close()
diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
index fcf39f6391..7a706ab256 100644
--- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
+++ b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala
@@ -236,16 +236,16 @@ class MesosFineGrainedSchedulerBackendSuite
mesosOffers.add(createOffer(3, minMem, minCpu))
val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2)
- expectedWorkerOffers.append(new WorkerOffer(
+ expectedWorkerOffers += new WorkerOffer(
mesosOffers.get(0).getSlaveId.getValue,
mesosOffers.get(0).getHostname,
(minCpu - backend.mesosExecutorCores).toInt
- ))
- expectedWorkerOffers.append(new WorkerOffer(
+ )
+ expectedWorkerOffers += new WorkerOffer(
mesosOffers.get(2).getSlaveId.getValue,
mesosOffers.get(2).getHostname,
(minCpu - backend.mesosExecutorCores).toInt
- ))
+ )
val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))
when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc)))
when(taskScheduler.CPUS_PER_TASK).thenReturn(2)
@@ -339,11 +339,11 @@ class MesosFineGrainedSchedulerBackendSuite
val backend = new MesosFineGrainedSchedulerBackend(taskScheduler, sc, "master")
val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](1)
- expectedWorkerOffers.append(new WorkerOffer(
+ expectedWorkerOffers += new WorkerOffer(
mesosOffers.get(0).getSlaveId.getValue,
mesosOffers.get(0).getHostname,
2 // Deducting 1 for executor
- ))
+ )
val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0)))
when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc)))
diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala
index 207f662e33..4d4b06b095 100644
--- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala
+++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala
@@ -991,7 +991,7 @@ object Matrices {
val data = new ArrayBuffer[(Int, Int, Double)]()
dnMat.foreachActive { (i, j, v) =>
if (v != 0.0) {
- data.append((i, j + startCol, v))
+ data += Tuple3(i, j + startCol, v)
}
}
startCol += nCols
@@ -1061,7 +1061,7 @@ object Matrices {
val data = new ArrayBuffer[(Int, Int, Double)]()
dnMat.foreachActive { (i, j, v) =>
if (v != 0.0) {
- data.append((i + startRow, j, v))
+ data += Tuple3(i + startRow, j, v)
}
}
startRow += nRows
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
index 8659cea4b8..6642999a21 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala
@@ -1128,7 +1128,7 @@ object Matrices {
val data = new ArrayBuffer[(Int, Int, Double)]()
dnMat.foreachActive { (i, j, v) =>
if (v != 0.0) {
- data.append((i, j + startCol, v))
+ data += Tuple3(i, j + startCol, v)
}
}
startCol += nCols
@@ -1198,7 +1198,7 @@ object Matrices {
val data = new ArrayBuffer[(Int, Int, Double)]()
dnMat.foreachActive { (i, j, v) =>
if (v != 0.0) {
- data.append((i + startRow, j, v))
+ data += Tuple3(i + startRow, j, v)
}
}
startRow += nRows
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala
index 9782350587..ff1068417d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala
@@ -257,7 +257,7 @@ class BlockMatrix @Since("1.3.0") (
val colStart = blockColIndex.toLong * colsPerBlock
val entryValues = new ArrayBuffer[MatrixEntry]()
mat.foreachActive { (i, j, v) =>
- if (v != 0.0) entryValues.append(new MatrixEntry(rowStart + i, colStart + j, v))
+ if (v != 0.0) entryValues += new MatrixEntry(rowStart + i, colStart + j, v)
}
entryValues
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
index f372355005..123e0bb3e6 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
@@ -252,7 +252,7 @@ object GradientDescent extends Logging {
* lossSum is computed using the weights from the previous iteration
* and regVal is the regularization value computed in the previous iteration as well.
*/
- stochasticLossHistory.append(lossSum / miniBatchSize + regVal)
+ stochasticLossHistory += lossSum / miniBatchSize + regVal
val update = updater.compute(
weights, Vectors.fromBreeze(gradientSum / miniBatchSize.toDouble),
stepSize, i, regParam)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala
index bf98bf2f5f..5f797a60f0 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala
@@ -95,7 +95,7 @@ class StreamingLogisticRegressionSuite extends SparkFunSuite with TestSuiteBase
// (we add a count to ensure the result is a DStream)
ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => {
model.trainOn(inputDStream)
- inputDStream.foreachRDD(x => history.append(math.abs(model.latestModel().weights(0) - B)))
+ inputDStream.foreachRDD(x => history += math.abs(model.latestModel().weights(0) - B))
inputDStream.count()
})
runStreams(ssc, numBatches, numBatches)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala
index 34c07ed170..eaeaa3fc1e 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala
@@ -109,7 +109,7 @@ class StreamingLinearRegressionSuite extends SparkFunSuite with TestSuiteBase {
// (we add a count to ensure the result is a DStream)
ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => {
model.trainOn(inputDStream)
- inputDStream.foreachRDD(x => history.append(math.abs(model.latestModel().weights(0) - 10.0)))
+ inputDStream.foreachRDD(x => history += math.abs(model.latestModel().weights(0) - 10.0))
inputDStream.count()
})
runStreams(ssc, numBatches, numBatches)
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(
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala
index 71f3304f1b..35f0166ed0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala
@@ -157,7 +157,7 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp
/** Write all the records in the buffer to the write ahead log. */
private def flushRecords(): Unit = {
try {
- buffer.append(walWriteQueue.take())
+ buffer += walWriteQueue.take()
val numBatched = walWriteQueue.drainTo(buffer.asJava) + 1
logDebug(s"Received $numBatched records from queue")
} catch {
diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
index 0daf1ea0bc..2f4b498b3c 100644
--- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -332,14 +332,14 @@ private[yarn] class YarnAllocator(
val newLocalityRequests = new mutable.ArrayBuffer[ContainerRequest]
containerLocalityPreferences.foreach {
case ContainerLocalityPreferences(nodes, racks) if nodes != null =>
- newLocalityRequests.append(createContainerRequest(resource, nodes, racks))
+ newLocalityRequests += createContainerRequest(resource, nodes, racks)
case _ =>
}
if (availableContainers >= newLocalityRequests.size) {
// more containers are available than needed for locality, fill in requests for any host
for (i <- 0 until (availableContainers - newLocalityRequests.size)) {
- newLocalityRequests.append(createContainerRequest(resource, null, null))
+ newLocalityRequests += createContainerRequest(resource, null, null)
}
} else {
val numToCancel = newLocalityRequests.size - availableContainers