aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorSean Owen <sowen@cloudera.com>2016-03-03 09:54:09 +0000
committerSean Owen <sowen@cloudera.com>2016-03-03 09:54:09 +0000
commite97fc7f176f8bf501c9b3afd8410014e3b0e1602 (patch)
tree23a11a3646b13195aaf50078a0f35fad96190618
parent02b7677e9584f5ccd68869abdb0bf980dc847ce1 (diff)
downloadspark-e97fc7f176f8bf501c9b3afd8410014e3b0e1602.tar.gz
spark-e97fc7f176f8bf501c9b3afd8410014e3b0e1602.tar.bz2
spark-e97fc7f176f8bf501c9b3afd8410014e3b0e1602.zip
[SPARK-13423][WIP][CORE][SQL][STREAMING] Static analysis fixes for 2.x
## What changes were proposed in this pull request? Make some cross-cutting code improvements according to static analysis. These are individually up for discussion since they exist in separate commits that can be reverted. The changes are broadly: - Inner class should be static - Mismatched hashCode/equals - Overflow in compareTo - Unchecked warnings - Misuse of assert, vs junit.assert - get(a) + getOrElse(b) -> getOrElse(a,b) - Array/String .size -> .length (occasionally, -> .isEmpty / .nonEmpty) to avoid implicit conversions - Dead code - tailrec - exists(_ == ) -> contains find + nonEmpty -> exists filter + size -> count - reduce(_+_) -> sum map + flatten -> map The most controversial may be .size -> .length simply because of its size. It is intended to avoid implicits that might be expensive in some places. ## How was the this patch tested? Existing Jenkins unit tests. Author: Sean Owen <sowen@cloudera.com> Closes #11292 from srowen/SPARK-13423.
-rw-r--r--common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java2
-rw-r--r--common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java12
-rw-r--r--common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java2
-rw-r--r--common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java7
-rw-r--r--common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java6
-rw-r--r--common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java8
-rw-r--r--common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java2
-rw-r--r--common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java2
-rw-r--r--common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala2
-rw-r--r--core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java4
-rw-r--r--core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java2
-rw-r--r--core/src/main/scala/org/apache/spark/Dependency.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/Partitioner.scala8
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/TaskEndReason.scala1
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala10
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala2
-rwxr-xr-xcore/src/main/scala/org/apache/spark/deploy/worker/Worker.scala7
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/StorageUtils.scala6
-rw-r--r--core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala2
-rw-r--r--core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java36
-rw-r--r--core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java2
-rw-r--r--core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java6
-rw-r--r--core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java22
-rw-r--r--core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java3
-rw-r--r--core/src/test/scala/org/apache/spark/AccumulatorSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/SparkConfSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala12
-rw-r--r--core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala6
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala2
-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/SparkConfWithEnv.scala4
-rw-r--r--examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java4
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala4
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/SparkALS.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala2
-rw-r--r--external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala2
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala2
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala6
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala10
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala16
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala2
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala5
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala4
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala2
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala2
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala14
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala6
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala6
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala12
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala10
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala2
-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/linalg/distributed/IndexedRowMatrix.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala8
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala4
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java4
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala2
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala2
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala10
-rw-r--r--sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java1
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala3
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala10
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala2
-rw-r--r--sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java16
-rw-r--r--sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java21
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala2
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala15
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala4
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala5
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala6
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala4
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala2
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala2
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala2
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala8
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala8
-rw-r--r--yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala2
147 files changed, 345 insertions, 293 deletions
diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java
index 70c849d60e..d17e986e17 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java
@@ -132,7 +132,7 @@ public class ChunkFetchIntegrationSuite {
testFile.delete();
}
- class FetchResult {
+ static class FetchResult {
public Set<Integer> successChunks;
public Set<Integer> failedChunks;
public List<ManagedBuffer> buffers;
diff --git a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java
index f9b5bf96d6..e2d026c66f 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java
@@ -124,8 +124,8 @@ public class RequestTimeoutIntegrationSuite {
synchronized (callback1) {
client.sendRpc(ByteBuffer.allocate(0), callback1);
callback1.wait(4 * 1000);
- assert (callback1.failure != null);
- assert (callback1.failure instanceof IOException);
+ assertNotNull(callback1.failure);
+ assertTrue(callback1.failure instanceof IOException);
}
semaphore.release();
}
@@ -167,8 +167,8 @@ public class RequestTimeoutIntegrationSuite {
synchronized (callback0) {
client0.sendRpc(ByteBuffer.allocate(0), callback0);
callback0.wait(FOREVER);
- assert (callback0.failure instanceof IOException);
- assert (!client0.isActive());
+ assertTrue(callback0.failure instanceof IOException);
+ assertFalse(client0.isActive());
}
// Increment the semaphore and the second request should succeed quickly.
@@ -236,7 +236,7 @@ public class RequestTimeoutIntegrationSuite {
synchronized (callback1) {
// failed at same time as previous
- assert (callback0.failure instanceof IOException);
+ assertTrue(callback0.failure instanceof IOException);
}
}
@@ -244,7 +244,7 @@ public class RequestTimeoutIntegrationSuite {
* Callback which sets 'success' or 'failure' on completion.
* Additionally notifies all waiters on this callback when invoked.
*/
- class TestCallback implements RpcResponseCallback, ChunkReceivedCallback {
+ static class TestCallback implements RpcResponseCallback, ChunkReceivedCallback {
int successLength = -1;
Throwable failure;
diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
index 9e9be98c14..a7a99f3bfc 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
@@ -91,7 +91,7 @@ public class RpcIntegrationSuite {
clientFactory.close();
}
- class RpcResult {
+ static class RpcResult {
public Set<String> successMessages;
public Set<String> errorMessages;
}
diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java
index dac7d4a5b0..9a89dd114f 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger;
import com.google.common.collect.Maps;
import org.junit.After;
+import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -95,7 +96,7 @@ public class TransportClientFactorySuite {
try {
TransportClient client =
factory.createClient(TestUtils.getLocalHost(), server1.getPort());
- assert (client.isActive());
+ assertTrue(client.isActive());
clients.add(client);
} catch (IOException e) {
failed.incrementAndGet();
@@ -115,8 +116,8 @@ public class TransportClientFactorySuite {
attempts[i].join();
}
- assert(failed.get() == 0);
- assert(clients.size() == maxConnections);
+ Assert.assertEquals(0, failed.get());
+ Assert.assertEquals(clients.size(), maxConnections);
for (TransportClient client : clients) {
client.close();
diff --git a/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java
index fbbe4b7014..b341c5681e 100644
--- a/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java
+++ b/common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java
@@ -65,7 +65,7 @@ public class MessageWithHeaderSuite {
assertEquals(42, result.readLong());
assertEquals(84, result.readLong());
- assert(msg.release());
+ assertTrue(msg.release());
assertEquals(0, bodyPassedToNettyManagedBuffer.refCnt());
assertEquals(0, header.refCnt());
}
@@ -77,7 +77,7 @@ public class MessageWithHeaderSuite {
ByteBuf body = (ByteBuf) managedBuf.convertToNetty();
assertEquals(2, body.refCnt());
MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes());
- assert(msg.release());
+ assertTrue(msg.release());
Mockito.verify(managedBuf, Mockito.times(1)).release();
assertEquals(0, body.refCnt());
}
@@ -94,7 +94,7 @@ public class MessageWithHeaderSuite {
for (long i = 0; i < 8; i++) {
assertEquals(i, result.readLong());
}
- assert(msg.release());
+ assertTrue(msg.release());
}
private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception {
diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java
index 94a61d6caa..eeb0019411 100644
--- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java
+++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java
@@ -56,6 +56,14 @@ public class RegisterDriver extends BlockTransferMessage {
return Objects.hashCode(appId);
}
+ @Override
+ public boolean equals(Object o) {
+ if (!(o instanceof RegisterDriver)) {
+ return false;
+ }
+ return Objects.equal(appId, ((RegisterDriver) o).appId);
+ }
+
public static RegisterDriver decode(ByteBuf buf) {
String appId = Encoders.Strings.decode(buf);
return new RegisterDriver(appId);
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
index 5e706bf401..ecbbe7bfa3 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
@@ -109,7 +109,7 @@ public class ExternalShuffleIntegrationSuite {
handler.applicationRemoved(APP_ID, false /* cleanupLocalDirs */);
}
- class FetchResult {
+ static class FetchResult {
public Set<String> successBlocks;
public Set<String> failedBlocks;
public List<ManagedBuffer> buffers;
diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
index 3a6ef0d3f8..91882e3b3b 100644
--- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
+++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
@@ -305,7 +305,7 @@ public class RetryingBlockFetcherSuite {
}
}
- assert stub != null;
+ assertNotNull(stub);
stub.when(fetchStarter).createAndStart((String[]) any(), (BlockFetchingListener) anyObject());
String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]);
new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start();
diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala
index b3bbd68827..8a6b9e3e45 100644
--- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala
+++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala
@@ -193,7 +193,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty
test("concat") {
def concat(orgin: Seq[String]): String =
- if (orgin.exists(_ == null)) null else orgin.mkString
+ if (orgin.contains(null)) null else orgin.mkString
forAll { (inputs: Seq[String]) =>
assert(UTF8String.concat(inputs.map(toUTF8): _*) === toUTF8(inputs.mkString))
diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
index d74602cd20..2381cff61f 100644
--- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java
@@ -30,7 +30,9 @@ final class ShuffleInMemorySorter {
private static final class SortComparator implements Comparator<PackedRecordPointer> {
@Override
public int compare(PackedRecordPointer left, PackedRecordPointer right) {
- return left.getPartitionId() - right.getPartitionId();
+ int leftId = left.getPartitionId();
+ int rightId = right.getPartitionId();
+ return leftId < rightId ? -1 : (leftId > rightId ? 1 : 0);
}
}
private static final SortComparator SORT_COMPARATOR = new SortComparator();
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index 296bf722fc..9236bd2c04 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -550,7 +550,7 @@ public final class UnsafeExternalSorter extends MemoryConsumer {
/**
* Chain multiple UnsafeSorterIterator together as single one.
*/
- class ChainedIterator extends UnsafeSorterIterator {
+ static class ChainedIterator extends UnsafeSorterIterator {
private final Queue<UnsafeSorterIterator> iterators;
private UnsafeSorterIterator current;
diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
index 9aafc9eb1c..b65cfdc4df 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -88,7 +88,7 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag](
val shuffleId: Int = _rdd.context.newShuffleId()
val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle(
- shuffleId, _rdd.partitions.size, this)
+ shuffleId, _rdd.partitions.length, this)
_rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this))
}
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index a7c2790c83..976c19f2b0 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -55,14 +55,14 @@ object Partitioner {
* We use two method parameters (rdd, others) to enforce callers passing at least 1 RDD.
*/
def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = {
- val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.size).reverse
+ val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.length).reverse
for (r <- bySize if r.partitioner.isDefined && r.partitioner.get.numPartitions > 0) {
return r.partitioner.get
}
if (rdd.context.conf.contains("spark.default.parallelism")) {
new HashPartitioner(rdd.context.defaultParallelism)
} else {
- new HashPartitioner(bySize.head.partitions.size)
+ new HashPartitioner(bySize.head.partitions.length)
}
}
}
@@ -122,7 +122,7 @@ class RangePartitioner[K : Ordering : ClassTag, V](
// This is the sample size we need to have roughly balanced output partitions, capped at 1M.
val sampleSize = math.min(20.0 * partitions, 1e6)
// Assume the input partitions are roughly balanced and over-sample a little bit.
- val sampleSizePerPartition = math.ceil(3.0 * sampleSize / rdd.partitions.size).toInt
+ val sampleSizePerPartition = math.ceil(3.0 * sampleSize / rdd.partitions.length).toInt
val (numItems, sketched) = RangePartitioner.sketch(rdd.map(_._1), sampleSizePerPartition)
if (numItems == 0L) {
Array.empty
@@ -137,7 +137,7 @@ class RangePartitioner[K : Ordering : ClassTag, V](
imbalancedPartitions += idx
} else {
// The weight is 1 over the sampling probability.
- val weight = (n.toDouble / sample.size).toFloat
+ val weight = (n.toDouble / sample.length).toFloat
for (key <- sample) {
candidates += ((key, weight))
}
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 0e8b735b92..b503c6184a 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -23,8 +23,8 @@ import java.net.URI
import java.util.{Arrays, Properties, UUID}
import java.util.concurrent.ConcurrentMap
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReference}
-import java.util.UUID.randomUUID
+import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.collection.Map
import scala.collection.generic.Growable
@@ -391,8 +391,8 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
_conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER)
- _jars = _conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten
- _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0))
+ _jars = _conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten
+ _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.nonEmpty))
.toSeq.flatten
_eventLogDir =
@@ -2310,6 +2310,7 @@ object SparkContext extends Logging {
* Create a task scheduler based on a given master URL.
* Return a 2-tuple of the scheduler backend and the task scheduler.
*/
+ @tailrec
private def createTaskScheduler(
sc: SparkContext,
master: String,
diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index c8f201ea9e..509fb2eb0e 100644
--- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala
+++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
@@ -248,7 +248,6 @@ case class ExecutorLostFailure(
} else {
"unrelated to the running tasks"
}
- s"ExecutorLostFailure (executor ${execId} exited due to an issue ${exitBehavior})"
s"ExecutorLostFailure (executor ${execId} exited ${exitBehavior})" +
reason.map { r => s" Reason: $r" }.getOrElse("")
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index 255420182b..a86ee66fb7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -19,6 +19,7 @@ package org.apache.spark.deploy
import java.net.{URI, URISyntaxException}
+import scala.annotation.tailrec
import scala.collection.mutable.ListBuffer
import org.apache.log4j.Level
@@ -49,6 +50,7 @@ private[deploy] class ClientArguments(args: Array[String]) {
parse(args.toList)
+ @tailrec
private def parse(args: List[String]): Unit = args match {
case ("--cores" | "-c") :: IntParam(value) :: tail =>
cores = value
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index d5a3383932..7d7ddccdcf 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -22,6 +22,7 @@ import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowab
import java.net.URL
import java.security.PrivilegedExceptionAction
+import scala.annotation.tailrec
import scala.collection.mutable.{ArrayBuffer, HashMap, Map}
import org.apache.commons.lang3.StringUtils
@@ -150,6 +151,7 @@ object SparkSubmit {
* Second, we use this launch environment to invoke the main method of the child
* main class.
*/
+ @tailrec
private def submit(args: SparkSubmitArguments): Unit = {
val (childArgs, childClasspath, sysProps, childMainClass) = prepareSubmitEnvironment(args)
@@ -721,6 +723,7 @@ object SparkSubmit {
throw new IllegalStateException("The main method in the given main class must be static")
}
+ @tailrec
def findCause(t: Throwable): Throwable = t match {
case e: UndeclaredThrowableException =>
if (e.getCause() != null) findCause(e.getCause()) else e
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
index d03bab3820..fc3790f8d7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala
@@ -17,6 +17,8 @@
package org.apache.spark.deploy.history
+import scala.annotation.tailrec
+
import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.util.Utils
@@ -29,6 +31,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin
parse(args.toList)
+ @tailrec
private def parse(args: List[String]): Unit = {
if (args.length == 1) {
setLogDirectory(args.head)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
index 44cefbc77f..9cd7458ba0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala
@@ -17,6 +17,8 @@
package org.apache.spark.deploy.master
+import scala.annotation.tailrec
+
import org.apache.spark.SparkConf
import org.apache.spark.util.{IntParam, Utils}
@@ -49,6 +51,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) {
webUiPort = conf.get("spark.master.ui.port").toInt
}
+ @tailrec
private def parse(args: List[String]): Unit = args match {
case ("--ip" | "-i") :: value :: tail =>
Utils.checkHost(value, "ip no longer supported, please use hostname " + value)
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
index 39b2647a90..fb07c39dd0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala
@@ -32,7 +32,7 @@ private[spark] class MasterSource(val master: Master) extends Source {
// Gauge for alive worker numbers in cluster
metricRegistry.register(MetricRegistry.name("aliveWorkers"), new Gauge[Int]{
- override def getValue: Int = master.workers.filter(_.state == WorkerState.ALIVE).size
+ override def getValue: Int = master.workers.count(_.state == WorkerState.ALIVE)
})
// Gauge for application numbers in cluster
@@ -42,6 +42,6 @@ private[spark] class MasterSource(val master: Master) extends Source {
// Gauge for waiting application numbers in cluster
metricRegistry.register(MetricRegistry.name("waitingApps"), new Gauge[Int] {
- override def getValue: Int = master.apps.filter(_.state == ApplicationState.WAITING).size
+ override def getValue: Int = master.apps.count(_.state == ApplicationState.WAITING)
})
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index 540e802420..b0cedef72e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -50,7 +50,7 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer
override def read[T: ClassTag](prefix: String): Seq[T] = {
zk.getChildren.forPath(WORKING_DIR).asScala
- .filter(_.startsWith(prefix)).map(deserializeFromFile[T]).flatten
+ .filter(_.startsWith(prefix)).flatMap(deserializeFromFile[T])
}
override def close() {
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
index f9b0279c3d..363f4b84f8 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -107,18 +107,18 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
</li>
}.getOrElse { Seq.empty }
}
- <li><strong>Alive Workers:</strong> {aliveWorkers.size}</li>
+ <li><strong>Alive Workers:</strong> {aliveWorkers.length}</li>
<li><strong>Cores in use:</strong> {aliveWorkers.map(_.cores).sum} Total,
{aliveWorkers.map(_.coresUsed).sum} Used</li>
<li><strong>Memory in use:</strong>
{Utils.megabytesToString(aliveWorkers.map(_.memory).sum)} Total,
{Utils.megabytesToString(aliveWorkers.map(_.memoryUsed).sum)} Used</li>
<li><strong>Applications:</strong>
- {state.activeApps.size} Running,
- {state.completedApps.size} Completed </li>
+ {state.activeApps.length} Running,
+ {state.completedApps.length} Completed </li>
<li><strong>Drivers:</strong>
- {state.activeDrivers.size} Running,
- {state.completedDrivers.size} Completed </li>
+ {state.activeDrivers.length} Running,
+ {state.completedDrivers.length} Completed </li>
<li><strong>Status:</strong> {state.status}</li>
</ul>
</div>
diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
index 5accaf78d0..38935e3209 100644
--- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala
@@ -17,6 +17,8 @@
package org.apache.spark.deploy.mesos
+import scala.annotation.tailrec
+
import org.apache.spark.SparkConf
import org.apache.spark.util.{IntParam, Utils}
@@ -34,6 +36,7 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf:
propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile)
+ @tailrec
private def parse(args: List[String]): Unit = args match {
case ("--host" | "-h") :: value :: tail =>
Utils.checkHost(value, "Please use hostname " + value)
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala
index 4ec6bfe2f9..006e2e1472 100644
--- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala
@@ -382,7 +382,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging {
logWarning(s"Unable to connect to server ${masterUrl}.")
lostMasters += masterUrl
}
- lostMasters.size >= masters.size
+ lostMasters.size >= masters.length
}
}
@@ -412,13 +412,13 @@ private[spark] object RestSubmissionClient {
}
def main(args: Array[String]): Unit = {
- if (args.size < 2) {
+ if (args.length < 2) {
sys.error("Usage: RestSubmissionClient [app resource] [main class] [app args*]")
sys.exit(1)
}
val appResource = args(0)
val mainClass = args(1)
- val appArgs = args.slice(2, args.size)
+ val appArgs = args.slice(2, args.length)
val conf = new SparkConf
val env = filterSystemEnvironment(sys.env)
run(appResource, mainClass, appArgs, conf, env)
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
index a8b2f78889..3b96488a12 100644
--- a/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala
@@ -94,7 +94,7 @@ private[mesos] class MesosSubmitRequestServlet(
val driverCores = sparkProperties.get("spark.driver.cores")
val appArgs = request.appArgs
val environmentVariables = request.environmentVariables
- val name = request.sparkProperties.get("spark.app.name").getOrElse(mainClass)
+ val name = request.sparkProperties.getOrElse("spark.app.name", mainClass)
// Construct driver description
val conf = new SparkConf(false).setAll(sparkProperties)
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index df3c286a0a..1c24c631ee 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -148,7 +148,7 @@ private[deploy] class Worker(
// time so that we can register with all masters.
private val registerMasterThreadPool = ThreadUtils.newDaemonCachedThreadPool(
"worker-register-master-threadpool",
- masterRpcAddresses.size // Make sure we can register with all masters at the same time
+ masterRpcAddresses.length // Make sure we can register with all masters at the same time
)
var coresUsed = 0
@@ -445,13 +445,12 @@ private[deploy] class Worker(
// Create local dirs for the executor. These are passed to the executor via the
// SPARK_EXECUTOR_DIRS environment variable, and deleted by the Worker when the
// application finishes.
- val appLocalDirs = appDirectories.get(appId).getOrElse {
+ val appLocalDirs = appDirectories.getOrElse(appId,
Utils.getOrCreateLocalRootDirs(conf).map { dir =>
val appDir = Utils.createDirectory(dir, namePrefix = "executor")
Utils.chmod700(appDir)
appDir.getAbsolutePath()
- }.toSeq
- }
+ }.toSeq)
appDirectories(appId) = appLocalDirs
val manager = new ExecutorRunner(
appId,
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
index de3c7cd265..391eb41190 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala
@@ -19,6 +19,8 @@ package org.apache.spark.deploy.worker
import java.lang.management.ManagementFactory
+import scala.annotation.tailrec
+
import org.apache.spark.util.{IntParam, MemoryParam, Utils}
import org.apache.spark.SparkConf
@@ -63,6 +65,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) {
checkWorkerMemory()
+ @tailrec
private def parse(args: List[String]): Unit = args match {
case ("--ip" | "-i") :: value :: tail =>
Utils.checkHost(value, "ip no longer supported, please use hostname " + value)
diff --git a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala
index d25452daf7..b089bbd7e9 100644
--- a/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala
+++ b/core/src/main/scala/org/apache/spark/partial/ApproximateActionListener.scala
@@ -38,7 +38,7 @@ private[spark] class ApproximateActionListener[T, U, R](
extends JobListener {
val startTime = System.currentTimeMillis()
- val totalTasks = rdd.partitions.size
+ val totalTasks = rdd.partitions.length
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/org/apache/spark/rdd/LocalCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala
index bfe19195fc..a163bbd264 100644
--- a/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/LocalCheckpointRDD.scala
@@ -41,7 +41,7 @@ private[spark] class LocalCheckpointRDD[T: ClassTag](
extends CheckpointRDD[T](sc) {
def this(rdd: RDD[T]) {
- this(rdd.context, rdd.id, rdd.partitions.size)
+ this(rdd.context, rdd.id, rdd.partitions.length)
}
protected override def getPartitions: Array[Partition] = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index ba773e1e7b..e2eaef5ec4 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -22,6 +22,7 @@ import java.util.Properties
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicInteger
+import scala.annotation.tailrec
import scala.collection.Map
import scala.collection.mutable.{HashMap, HashSet, Stack}
import scala.concurrent.Await
@@ -469,6 +470,7 @@ class DAGScheduler(
* all of that stage's ancestors.
*/
private def updateJobIdStageIdMaps(jobId: Int, stage: Stage): Unit = {
+ @tailrec
def updateJobIdStageIdMapsList(stages: List[Stage]) {
if (stages.nonEmpty) {
val s = stages.head
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 29341dfe30..8b2f4973ef 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -620,7 +620,7 @@ private[spark] object TaskSchedulerImpl {
while (found) {
found = false
for (key <- keyList) {
- val containerList: ArrayBuffer[T] = map.get(key).getOrElse(null)
+ val containerList: ArrayBuffer[T] = map.getOrElse(key, null)
assert(containerList != null)
// Get the index'th entry for this host - if present
if (index < containerList.size){
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 2b0eab7169..f1339d530a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -202,7 +202,7 @@ private[spark] class TaskSetManager(
", but there are no executors alive there.")
}
}
- case _ => Unit
+ case _ =>
}
pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index
for (rack <- sched.getRackForHost(loc.host)) {
@@ -828,7 +828,7 @@ private[spark] class TaskSetManager(
val time = clock.getTimeMillis()
val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray
Arrays.sort(durations)
- val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.size - 1))
+ val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.length - 1))
val threshold = max(SPECULATION_MULTIPLIER * medianDuration, 100)
// TODO: Threshold should also look at standard deviation of task durations and have a lower
// bound based on that.
diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index ea718a0edb..8b72da2ee0 100644
--- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
@@ -68,7 +68,7 @@ private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoa
// scalastyle:on classforname
} catch {
case e: ClassNotFoundException =>
- JavaDeserializationStream.primitiveMappings.get(desc.getName).getOrElse(throw e)
+ JavaDeserializationStream.primitiveMappings.getOrElse(desc.getName, throw e)
}
}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
index 7750a09623..5c03609e5e 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllRDDResource.scala
@@ -61,7 +61,7 @@ private[spark] object AllRDDResource {
.flatMap { _.rddBlocksById(rddId) }
.sortWith { _._1.name < _._1.name }
.map { case (blockId, status) =>
- (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown")))
+ (blockId, status, blockLocations.getOrElse(blockId, Seq[String]("Unknown")))
}
val dataDistribution = if (includeDetails) {
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
index a0f6360bc5..653150385c 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneJobResource.scala
@@ -30,7 +30,7 @@ private[v1] class OneJobResource(ui: SparkUI) {
def oneJob(@PathParam("jobId") jobId: Int): JobData = {
val statusToJobs: Seq[(JobExecutionStatus, Seq[JobUIData])] =
AllJobsResource.getStatusToJobs(ui)
- val jobOpt = statusToJobs.map {_._2} .flatten.find { jobInfo => jobInfo.jobId == jobId}
+ val jobOpt = statusToJobs.flatMap(_._2).find { jobInfo => jobInfo.jobId == jobId}
jobOpt.map { job =>
AllJobsResource.convertJobData(job, ui.jobProgressListener, false)
}.getOrElse {
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
index 8e2cfb2441..43cd15921c 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala
@@ -82,9 +82,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) {
def rddBlocks: Map[BlockId, BlockStatus] = _rddBlocks.flatMap { case (_, blocks) => blocks }
/** Return the blocks that belong to the given RDD stored in this block manager. */
- def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = {
- _rddBlocks.get(rddId).getOrElse(Map.empty)
- }
+ def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = _rddBlocks.getOrElse(rddId, Map.empty)
/** Add the given block to this storage status. If it already exists, overwrite it. */
private[spark] def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = {
@@ -143,7 +141,7 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) {
def getBlock(blockId: BlockId): Option[BlockStatus] = {
blockId match {
case RDDBlockId(rddId, _) =>
- _rddBlocks.get(rddId).map(_.get(blockId)).flatten
+ _rddBlocks.get(rddId).flatMap(_.get(blockId))
case _ =>
_nonRddBlocks.get(blockId)
}
diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
index 77c0bc8b53..f157a451ef 100644
--- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
+++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala
@@ -63,7 +63,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging {
return
}
val stageIds = sc.statusTracker.getActiveStageIds()
- val stages = stageIds.map(sc.statusTracker.getStageInfo).flatten.filter(_.numTasks() > 1)
+ val stages = stageIds.flatMap(sc.statusTracker.getStageInfo).filter(_.numTasks() > 1)
.filter(now - _.submissionTime() > FIRST_DELAY).sortBy(_.stageId())
if (stages.length > 0) {
show(now, stages.take(3)) // display at most 3 stages in same time
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
index 77ca60b000..2fd630a85c 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala
@@ -29,7 +29,7 @@ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") {
val operationGraphListener = parent.operationGraphListener
def isFairScheduler: Boolean =
- jobProgresslistener.schedulingMode.exists(_ == SchedulingMode.FAIR)
+ jobProgresslistener.schedulingMode.contains(SchedulingMode.FAIR)
attachPage(new AllJobsPage(this))
attachPage(new JobPage(this))
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
index 5989f0035b..ece5d0fce8 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala
@@ -34,7 +34,7 @@ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages"
attachPage(new StagePage(this))
attachPage(new PoolPage(this))
- def isFairScheduler: Boolean = progressListener.schedulingMode.exists(_ == SchedulingMode.FAIR)
+ def isFairScheduler: Boolean = progressListener.schedulingMode.contains(SchedulingMode.FAIR)
def handleKillRequest(request: HttpServletRequest): Unit = {
if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) {
diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala
index 89119cd357..bcae56e2f1 100644
--- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala
+++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala
@@ -52,9 +52,8 @@ private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListen
* An empty list is returned if one or more of its stages has been cleaned up.
*/
def getOperationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = synchronized {
- val skippedStageIds = jobIdToSkippedStageIds.get(jobId).getOrElse(Seq.empty)
- val graphs = jobIdToStageIds.get(jobId)
- .getOrElse(Seq.empty)
+ val skippedStageIds = jobIdToSkippedStageIds.getOrElse(jobId, Seq.empty)
+ val graphs = jobIdToStageIds.getOrElse(jobId, Seq.empty)
.flatMap { sid => stageIdToGraph.get(sid) }
// Mark any skipped stages as such
graphs.foreach { g =>
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
index c9bb49b83e..76d7c6d414 100644
--- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
@@ -156,7 +156,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") {
streamBlockTableSubrow(block._1, replications.head, replications.size, true)
} else {
streamBlockTableSubrow(block._1, replications.head, replications.size, true) ++
- replications.tail.map(streamBlockTableSubrow(block._1, _, replications.size, false)).flatten
+ replications.tail.flatMap(streamBlockTableSubrow(block._1, _, replications.size, false))
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index cfe247c668..9688cca4f0 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -27,6 +27,7 @@ import java.util.{Locale, Properties, Random, UUID}
import java.util.concurrent._
import javax.net.ssl.HttpsURLConnection
+import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.collection.Map
import scala.collection.mutable.ArrayBuffer
@@ -2219,6 +2220,7 @@ private[spark] object Utils extends Logging {
/**
* Return whether the specified file is a parent directory of the child file.
*/
+ @tailrec
def isInDirectory(parent: File, child: File): Boolean = {
if (child == null || parent == null) {
return false
diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala
index 050ece12f1..a0eb05c7c0 100644
--- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala
+++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala
@@ -117,7 +117,7 @@ private[spark] class RollingFileAppender(
}
}).sorted
val filesToBeDeleted = rolledoverFiles.take(
- math.max(0, rolledoverFiles.size - maxRetainedFiles))
+ math.max(0, rolledoverFiles.length - maxRetainedFiles))
filesToBeDeleted.foreach { file =>
logInfo(s"Deleting file executor log file ${file.getAbsolutePath}")
file.delete()
diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
index 776a2997cf..127789b632 100644
--- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
+++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
@@ -73,37 +73,37 @@ public class TaskMemoryManagerSuite {
TestMemoryConsumer c1 = new TestMemoryConsumer(manager);
TestMemoryConsumer c2 = new TestMemoryConsumer(manager);
c1.use(100);
- assert(c1.getUsed() == 100);
+ Assert.assertEquals(100, c1.getUsed());
c2.use(100);
- assert(c2.getUsed() == 100);
- assert(c1.getUsed() == 0); // spilled
+ Assert.assertEquals(100, c2.getUsed());
+ Assert.assertEquals(0, c1.getUsed()); // spilled
c1.use(100);
- assert(c1.getUsed() == 100);
- assert(c2.getUsed() == 0); // spilled
+ Assert.assertEquals(100, c1.getUsed());
+ Assert.assertEquals(0, c2.getUsed()); // spilled
c1.use(50);
- assert(c1.getUsed() == 50); // spilled
- assert(c2.getUsed() == 0);
+ Assert.assertEquals(50, c1.getUsed()); // spilled
+ Assert.assertEquals(0, c2.getUsed());
c2.use(50);
- assert(c1.getUsed() == 50);
- assert(c2.getUsed() == 50);
+ Assert.assertEquals(50, c1.getUsed());
+ Assert.assertEquals(50, c2.getUsed());
c1.use(100);
- assert(c1.getUsed() == 100);
- assert(c2.getUsed() == 0); // spilled
+ Assert.assertEquals(100, c1.getUsed());
+ Assert.assertEquals(0, c2.getUsed()); // spilled
c1.free(20);
- assert(c1.getUsed() == 80);
+ Assert.assertEquals(80, c1.getUsed());
c2.use(10);
- assert(c1.getUsed() == 80);
- assert(c2.getUsed() == 10);
+ Assert.assertEquals(80, c1.getUsed());
+ Assert.assertEquals(10, c2.getUsed());
c2.use(100);
- assert(c2.getUsed() == 100);
- assert(c1.getUsed() == 0); // spilled
+ Assert.assertEquals(100, c2.getUsed());
+ Assert.assertEquals(0, c1.getUsed()); // spilled
c1.free(0);
c2.free(100);
- assert(manager.cleanUpAllAllocatedMemory() == 0);
+ Assert.assertEquals(0, manager.cleanUpAllAllocatedMemory());
}
@Test
@@ -114,7 +114,7 @@ public class TaskMemoryManagerSuite {
.set("spark.unsafe.offHeap", "true")
.set("spark.memory.offHeap.size", "1000");
final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0);
- assert(manager.tungstenMemoryMode == MemoryMode.OFF_HEAP);
+ Assert.assertSame(MemoryMode.OFF_HEAP, manager.tungstenMemoryMode);
}
}
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java
index eb1da8e1b4..b4fa33f32a 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java
@@ -48,7 +48,7 @@ public class ShuffleInMemorySorterSuite {
public void testSortingEmptyInput() {
final ShuffleInMemorySorter sorter = new ShuffleInMemorySorter(consumer, 100);
final ShuffleInMemorySorter.ShuffleSorterIterator iter = sorter.getSortedIterator();
- assert(!iter.hasNext());
+ Assert.assertFalse(iter.hasNext());
}
@Test
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
index 876c3a2283..add9d937d3 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
@@ -139,7 +139,7 @@ public class UnsafeShuffleWriterSuite {
new Answer<InputStream>() {
@Override
public InputStream answer(InvocationOnMock invocation) throws Throwable {
- assert (invocation.getArguments()[0] instanceof TempShuffleBlockId);
+ assertTrue(invocation.getArguments()[0] instanceof TempShuffleBlockId);
InputStream is = (InputStream) invocation.getArguments()[1];
if (conf.getBoolean("spark.shuffle.compress", true)) {
return CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(is);
@@ -154,7 +154,7 @@ public class UnsafeShuffleWriterSuite {
new Answer<OutputStream>() {
@Override
public OutputStream answer(InvocationOnMock invocation) throws Throwable {
- assert (invocation.getArguments()[0] instanceof TempShuffleBlockId);
+ assertTrue(invocation.getArguments()[0] instanceof TempShuffleBlockId);
OutputStream os = (OutputStream) invocation.getArguments()[1];
if (conf.getBoolean("spark.shuffle.compress", true)) {
return CompressionCodec$.MODULE$.createCodec(conf).compressedOutputStream(os);
@@ -252,7 +252,7 @@ public class UnsafeShuffleWriterSuite {
createWriter(false).stop(false);
}
- class PandaException extends RuntimeException {
+ static class PandaException extends RuntimeException {
}
@Test(expected=PandaException.class)
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
index 32f5a1a7e6..492fe49ba4 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
@@ -323,23 +323,23 @@ public class UnsafeExternalSorterSuite {
record[0] = (long) i;
sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0);
}
- assert(sorter.getNumberOfAllocatedPages() >= 2);
+ assertTrue(sorter.getNumberOfAllocatedPages() >= 2);
UnsafeExternalSorter.SpillableIterator iter =
(UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator();
int lastv = 0;
for (int i = 0; i < n / 3; i++) {
iter.hasNext();
iter.loadNext();
- assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
+ assertTrue(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
lastv = i;
}
- assert(iter.spill() > 0);
- assert(iter.spill() == 0);
- assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == lastv);
+ assertTrue(iter.spill() > 0);
+ assertEquals(0, iter.spill());
+ assertTrue(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == lastv);
for (int i = n / 3; i < n; i++) {
iter.hasNext();
iter.loadNext();
- assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
+ assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
@@ -355,15 +355,15 @@ public class UnsafeExternalSorterSuite {
record[0] = (long) i;
sorter.insertRecord(record, Platform.LONG_ARRAY_OFFSET, recordSize, 0);
}
- assert(sorter.getNumberOfAllocatedPages() >= 2);
+ assertTrue(sorter.getNumberOfAllocatedPages() >= 2);
UnsafeExternalSorter.SpillableIterator iter =
(UnsafeExternalSorter.SpillableIterator) sorter.getSortedIterator();
- assert(iter.spill() > 0);
- assert(iter.spill() == 0);
+ assertTrue(iter.spill() > 0);
+ assertEquals(0, iter.spill());
for (int i = 0; i < n; i++) {
iter.hasNext();
iter.loadNext();
- assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
+ assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
@@ -394,7 +394,7 @@ public class UnsafeExternalSorterSuite {
for (int i = 0; i < n; i++) {
iter.hasNext();
iter.loadNext();
- assert(Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()) == i);
+ assertEquals(i, Platform.getLong(iter.getBaseObject(), iter.getBaseOffset()));
}
sorter.cleanupResources();
assertSpillFilesWereCleanedUp();
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
index 8e557ec0ab..ff41768df1 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
@@ -19,6 +19,7 @@ package org.apache.spark.util.collection.unsafe.sort;
import java.util.Arrays;
+import org.junit.Assert;
import org.junit.Test;
import org.apache.spark.HashPartitioner;
@@ -54,7 +55,7 @@ public class UnsafeInMemorySorterSuite {
mock(PrefixComparator.class),
100);
final UnsafeSorterIterator iter = sorter.getSortedIterator();
- assert(!iter.hasNext());
+ Assert.assertFalse(iter.hasNext());
}
@Test
diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
index 8acd0439b6..4ff8ae57ab 100644
--- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
@@ -347,7 +347,7 @@ private class SaveInfoListener extends SparkListener {
def getCompletedStageInfos: Seq[StageInfo] = completedStageInfos.toArray.toSeq
def getCompletedTaskInfos: Seq[TaskInfo] = completedTaskInfos.values.flatten.toSeq
def getCompletedTaskInfos(stageId: StageId, stageAttemptId: StageAttemptId): Seq[TaskInfo] =
- completedTaskInfos.get((stageId, stageAttemptId)).getOrElse(Seq.empty[TaskInfo])
+ completedTaskInfos.getOrElse((stageId, stageAttemptId), Seq.empty[TaskInfo])
/**
* If `jobCompletionCallback` is set, block until the next call has finished.
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index 2fe99e3f81..79881f30b2 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -237,7 +237,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst
conf.set(newName, "4")
assert(conf.get(newName) === "4")
- val count = conf.getAll.filter { case (k, v) => k.startsWith("spark.history.") }.size
+ val count = conf.getAll.count { case (k, v) => k.startsWith("spark.history.") }
assert(count === 4)
conf.set("spark.yarn.applicationMaster.waitTries", "42")
diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala
index e7cc1617cd..31ce9483cf 100644
--- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala
@@ -101,7 +101,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers {
val data = 1 until 100
val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
- assert(slices.map(_.size).reduceLeft(_ + _) === 99)
+ assert(slices.map(_.size).sum === 99)
assert(slices.forall(_.isInstanceOf[Range]))
}
@@ -109,7 +109,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers {
val data = 1 to 100
val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
- assert(slices.map(_.size).reduceLeft(_ + _) === 100)
+ assert(slices.map(_.size).sum === 100)
assert(slices.forall(_.isInstanceOf[Range]))
}
@@ -202,7 +202,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers {
val data = 1L until 100L
val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
- assert(slices.map(_.size).reduceLeft(_ + _) === 99)
+ assert(slices.map(_.size).sum === 99)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
}
@@ -210,7 +210,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers {
val data = 1L to 100L
val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
- assert(slices.map(_.size).reduceLeft(_ + _) === 100)
+ assert(slices.map(_.size).sum === 100)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
}
@@ -218,7 +218,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers {
val data = 1.0 until 100.0 by 1.0
val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
- assert(slices.map(_.size).reduceLeft(_ + _) === 99)
+ assert(slices.map(_.size).sum === 99)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
}
@@ -226,7 +226,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers {
val data = 1.0 to 100.0 by 1.0
val slices = ParallelCollectionRDD.slice(data, 3)
assert(slices.size === 3)
- assert(slices.map(_.size).reduceLeft(_ + _) === 100)
+ assert(slices.map(_.size).sum === 100)
assert(slices.forall(_.isInstanceOf[NumericRange[_]]))
}
diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
index 80347b800a..24daedab20 100644
--- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala
@@ -54,16 +54,16 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext {
assert(!nums.isEmpty())
assert(nums.max() === 4)
assert(nums.min() === 1)
- val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _)))
+ val partitionSums = nums.mapPartitions(iter => Iterator(iter.sum))
assert(partitionSums.collect().toList === List(3, 7))
val partitionSumsWithSplit = nums.mapPartitionsWithIndex {
- case(split, iter) => Iterator((split, iter.reduceLeft(_ + _)))
+ case(split, iter) => Iterator((split, iter.sum))
}
assert(partitionSumsWithSplit.collect().toList === List((0, 3), (1, 7)))
val partitionSumsWithIndex = nums.mapPartitionsWithIndex {
- case(split, iter) => Iterator((split, iter.reduceLeft(_ + _)))
+ case(split, iter) => Iterator((split, iter.sum))
}
assert(partitionSumsWithIndex.collect().toList === List((0, 3), (1, 7)))
diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala
index 56e0f01b3b..759d52fca5 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala
@@ -79,7 +79,7 @@ class MapStatusSuite extends SparkFunSuite {
test("HighlyCompressedMapStatus: estimated size should be the average non-empty block size") {
val sizes = Array.tabulate[Long](3000) { i => i.toLong }
- val avg = sizes.sum / sizes.filter(_ != 0).length
+ val avg = sizes.sum / sizes.count(_ != 0)
val loc = BlockManagerId("a", "b", 10)
val status = MapStatus(loc, sizes)
val status1 = compressAndDecompressMapStatus(status)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
index b5385c11a9..935e280e60 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
@@ -243,7 +243,7 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local
val resAfter = captor.getValue
val resSizeBefore = resBefore.accumUpdates.find(_.name == Some(RESULT_SIZE)).flatMap(_.update)
val resSizeAfter = resAfter.accumUpdates.find(_.name == Some(RESULT_SIZE)).flatMap(_.update)
- assert(resSizeBefore.exists(_ == 0L))
+ assert(resSizeBefore.contains(0L))
assert(resSizeAfter.exists(_.toString.toLong > 0L))
}
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 fdacd8c9f5..cf9f9da1e6 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
@@ -166,7 +166,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte
writer.stop( /* success = */ true)
assert(temporaryFilesCreated.nonEmpty)
assert(writer.getPartitionLengths.sum === outputFile.length())
- assert(writer.getPartitionLengths.filter(_ == 0L).size === 4) // should be 4 zero length files
+ assert(writer.getPartitionLengths.count(_ == 0L) === 4) // should be 4 zero length files
assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted
val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics.get
assert(shuffleWriteMetrics.bytesWritten === outputFile.length())
diff --git a/core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala b/core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala
index ddd5edf4f7..0c8b8cfdd5 100644
--- a/core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala
+++ b/core/src/test/scala/org/apache/spark/util/SparkConfWithEnv.scala
@@ -23,9 +23,7 @@ import org.apache.spark.SparkConf
* Customized SparkConf that allows env variables to be overridden.
*/
class SparkConfWithEnv(env: Map[String, String]) extends SparkConf(false) {
- override def getenv(name: String): String = {
- env.get(name).getOrElse(super.getenv(name))
- }
+ override def getenv(name: String): String = env.getOrElse(name, super.getenv(name))
override def clone: SparkConf = {
new SparkConfWithEnv(env).setAll(getAll)
diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java
index cf774667f6..7bb70d0c06 100644
--- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java
@@ -60,7 +60,9 @@ class JavaSampleActorReceiver<T> extends JavaActorReceiver {
@Override
public void onReceive(Object msg) throws Exception {
- store((T) msg);
+ @SuppressWarnings("unchecked")
+ T msgT = (T) msg;
+ store(msgT);
}
@Override
diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
index b26db0b246..e37a3fa69d 100644
--- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala
@@ -88,7 +88,7 @@ object DFSReadWriteTest {
def runLocalWordCount(fileContents: List[String]): Int = {
fileContents.flatMap(_.split(" "))
.flatMap(_.split("\t"))
- .filter(_.size > 0)
+ .filter(_.nonEmpty)
.groupBy(w => w)
.mapValues(_.size)
.values
@@ -119,7 +119,7 @@ object DFSReadWriteTest {
val dfsWordCount = readFileRDD
.flatMap(_.split(" "))
.flatMap(_.split("\t"))
- .filter(_.size > 0)
+ .filter(_.nonEmpty)
.map(w => (w, 1))
.countByKey()
.values
diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
index bec61f3cd4..a2d59a1c95 100644
--- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala
@@ -26,7 +26,7 @@ import org.apache.spark.util.Utils
* test driver submission in the standalone scheduler. */
object DriverSubmissionTest {
def main(args: Array[String]) {
- if (args.size < 1) {
+ if (args.length < 1) {
println("Usage: DriverSubmissionTest <seconds-to-sleep>")
System.exit(0)
}
diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
index a797111dba..134c3d1d63 100644
--- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala
@@ -46,7 +46,7 @@ object MultiBroadcastTest {
val barr1 = sc.broadcast(arr1)
val barr2 = sc.broadcast(arr2)
val observedSizes: RDD[(Int, Int)] = sc.parallelize(1 to 10, slices).map { _ =>
- (barr1.value.size, barr2.value.size)
+ (barr1.value.length, barr2.value.length)
}
// Collect the small RDD so we can print the observed sizes locally.
observedSizes.collect().foreach(i => println(i))
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
index 69799b7c2b..4263680c6f 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala
@@ -58,7 +58,7 @@ object SparkALS {
}
def update(i: Int, m: RealVector, us: Array[RealVector], R: RealMatrix) : RealVector = {
- val U = us.size
+ val U = us.length
val F = us(0).getDimension
var XtX: RealMatrix = new Array2DRowRealMatrix(F, F)
var Xty: RealVector = new ArrayRealVector(F)
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
index 038b2fe611..e89d555884 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala
@@ -118,7 +118,7 @@ object LDAExample {
preprocess(sc, params.input, params.vocabSize, params.stopwordFile)
corpus.cache()
val actualCorpusSize = corpus.count()
- val actualVocabSize = vocabArray.size
+ val actualVocabSize = vocabArray.length
val preprocessElapsed = (System.nanoTime() - preprocessStart) / 1e9
println()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala
index c4e5e965b8..011db4fd0c 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala
@@ -79,7 +79,7 @@ object SampledRDDs {
val sampledRDD = examples.sample(withReplacement = true, fraction = fraction)
println(s" RDD.sample(): sample has ${sampledRDD.count()} examples")
val sampledArray = examples.takeSample(withReplacement = true, num = expectedSampleSize)
- println(s" RDD.takeSample(): sample has ${sampledArray.size} examples")
+ println(s" RDD.takeSample(): sample has ${sampledArray.length} examples")
println()
diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
index 4b43550a06..773a2e5fc2 100644
--- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
@@ -69,7 +69,7 @@ object PageViewStream {
.groupByKey()
val errorRatePerZipCode = statusesPerZipCode.map{
case(zip, statuses) =>
- val normalCount = statuses.filter(_ == 200).size
+ val normalCount = statuses.count(_ == 200)
val errorCount = statuses.size - normalCount
val errorRatio = errorCount.toFloat / statuses.size
if (errorRatio > 0.05) {
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
index 4eb1556458..475167aa70 100644
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala
@@ -79,7 +79,7 @@ class KafkaRDD[
.map(_.asInstanceOf[KafkaRDDPartition])
.filter(_.count > 0)
- if (num < 1 || nonEmptyPartitions.size < 1) {
+ if (num < 1 || nonEmptyPartitions.isEmpty) {
return new Array[R](0)
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index 3e8c385302..87f3bc31e6 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -284,7 +284,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
if (selectedVertices.count > 1) {
found = true
val collectedVertices = selectedVertices.collect()
- retVal = collectedVertices(Random.nextInt(collectedVertices.size))
+ retVal = collectedVertices(Random.nextInt(collectedVertices.length))
}
}
retVal
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
index 53a9f92b82..5a0c479bb4 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
@@ -276,7 +276,7 @@ object VertexRDD {
def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = {
val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match {
case Some(p) => vertices
- case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size))
+ case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.length))
}
val vertexPartitions = vPartitioned.mapPartitions(
iter => Iterator(ShippableVertexPartition(iter)),
@@ -317,7 +317,7 @@ object VertexRDD {
): VertexRDD[VD] = {
val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match {
case Some(p) => vertices
- case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size))
+ case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.length))
}
val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get)
val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) {
@@ -358,7 +358,7 @@ object VertexRDD {
Function.tupled(RoutingTablePartition.edgePartitionToMsgs)))
.setName("VertexRDD.createRoutingTables - vid2pid (aggregation)")
- val numEdgePartitions = edges.partitions.size
+ val numEdgePartitions = edges.partitions.length
vid2pid.partitionBy(vertexPartitioner).mapPartitions(
iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)),
preservesPartitioning = true)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
index ab021a252e..b1da781663 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
@@ -151,9 +151,9 @@ class EdgePartition[
* applied to each edge
*/
def map[ED2: ClassTag](f: Edge[ED] => ED2): EdgePartition[ED2, VD] = {
- val newData = new Array[ED2](data.size)
+ val newData = new Array[ED2](data.length)
val edge = new Edge[ED]()
- val size = data.size
+ val size = data.length
var i = 0
while (i < size) {
edge.srcId = srcIds(i)
@@ -179,13 +179,13 @@ class EdgePartition[
*/
def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2, VD] = {
// Faster than iter.toArray, because the expected size is known.
- val newData = new Array[ED2](data.size)
+ val newData = new Array[ED2](data.length)
var i = 0
while (iter.hasNext) {
newData(i) = iter.next()
i += 1
}
- assert(newData.size == i)
+ assert(newData.length == i)
this.withData(newData)
}
@@ -311,7 +311,7 @@ class EdgePartition[
*
* @return size of the partition
*/
- val size: Int = localSrcIds.size
+ val size: Int = localSrcIds.length
/** The number of unique source vertices in the partition. */
def indexSize: Int = index.size
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
index b122969b81..da3db3c4dc 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
@@ -38,9 +38,9 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla
val edgeArray = edges.trim().array
new Sorter(Edge.edgeArraySortDataFormat[ED])
.sort(edgeArray, 0, edgeArray.length, Edge.lexicographicOrdering)
- val localSrcIds = new Array[Int](edgeArray.size)
- val localDstIds = new Array[Int](edgeArray.size)
- val data = new Array[ED](edgeArray.size)
+ val localSrcIds = new Array[Int](edgeArray.length)
+ val localDstIds = new Array[Int](edgeArray.length)
+ val data = new Array[ED](edgeArray.length)
val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int]
val global2local = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int]
val local2global = new PrimitiveVector[VertexId]
@@ -52,7 +52,7 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla
var currSrcId: VertexId = edgeArray(0).srcId
var currLocalId = -1
var i = 0
- while (i < edgeArray.size) {
+ while (i < edgeArray.length) {
val srcId = edgeArray(i).srcId
val dstId = edgeArray(i).dstId
localSrcIds(i) = global2local.changeValue(srcId,
@@ -98,9 +98,9 @@ class ExistingEdgePartitionBuilder[
val edgeArray = edges.trim().array
new Sorter(EdgeWithLocalIds.edgeArraySortDataFormat[ED])
.sort(edgeArray, 0, edgeArray.length, EdgeWithLocalIds.lexicographicOrdering)
- val localSrcIds = new Array[Int](edgeArray.size)
- val localDstIds = new Array[Int](edgeArray.size)
- val data = new Array[ED](edgeArray.size)
+ val localSrcIds = new Array[Int](edgeArray.length)
+ val localDstIds = new Array[Int](edgeArray.length)
+ val data = new Array[ED](edgeArray.length)
val index = new GraphXPrimitiveKeyOpenHashMap[VertexId, Int]
// Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and
// adding them to the index
@@ -108,7 +108,7 @@ class ExistingEdgePartitionBuilder[
index.update(edgeArray(0).srcId, 0)
var currSrcId: VertexId = edgeArray(0).srcId
var i = 0
- while (i < edgeArray.size) {
+ while (i < edgeArray.length) {
localSrcIds(i) = edgeArray(i).localSrcId
localDstIds(i) = edgeArray(i).localDstId
data(i) = edgeArray(i).attr
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
index 6e153b7e80..98e082cc44 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala
@@ -45,7 +45,7 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] (
* partitioner that allows co-partitioning with `partitionsRDD`.
*/
override val partitioner =
- partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitions.size)))
+ partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitions.length)))
override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect()
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 699731b360..7903caa312 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -93,7 +93,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
}
override def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] = {
- partitionBy(partitionStrategy, edges.partitions.size)
+ partitionBy(partitionStrategy, edges.partitions.length)
}
override def partitionBy(
@@ -352,7 +352,8 @@ object GraphImpl {
edgeStorageLevel: StorageLevel,
vertexStorageLevel: StorageLevel): GraphImpl[VD, ED] = {
val edgesCached = edges.withTargetStorageLevel(edgeStorageLevel).cache()
- val vertices = VertexRDD.fromEdges(edgesCached, edgesCached.partitions.size, defaultVertexAttr)
+ val vertices =
+ VertexRDD.fromEdges(edgesCached, edgesCached.partitions.length, defaultVertexAttr)
.withTargetStorageLevel(vertexStorageLevel)
fromExistingRDDs(vertices, edgesCached)
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala
index 3fd76902af..13e25b43f6 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala
@@ -108,10 +108,10 @@ private[graphx]
class RoutingTablePartition(
private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) extends Serializable {
/** The maximum number of edge partitions this `RoutingTablePartition` is built to join with. */
- val numEdgePartitions: Int = routingTable.size
+ val numEdgePartitions: Int = routingTable.length
/** Returns the number of vertices that will be sent to the specified edge partition. */
- def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.size
+ def partitionSize(pid: PartitionID): Int = routingTable(pid)._1.length
/** Returns an iterator over all vertex ids stored in this `RoutingTablePartition`. */
def iterator: Iterator[VertexId] = routingTable.iterator.flatMap(_._1.iterator)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala
index 96d807f9f9..6dab465fb9 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala
@@ -28,7 +28,7 @@ private[graphx]
class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD])
extends Serializable {
def iterator: Iterator[(VertexId, VD)] =
- (0 until vids.size).iterator.map { i => (vids(i), attrs(i)) }
+ (0 until vids.length).iterator.map { i => (vids(i), attrs(i)) }
}
private[graphx]
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
index 51bcdf20de..026fb8bc7b 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
@@ -70,7 +70,7 @@ object TriangleCount {
graph.collectNeighborIds(EdgeDirection.Either).mapValues { (vid, nbrs) =>
val set = new VertexSet(nbrs.length)
var i = 0
- while (i < nbrs.size) {
+ while (i < nbrs.length) {
// prevent self cycle
if (nbrs(i) != vid) {
set.add(nbrs(i))
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
index f1aa685a79..0bb9e0a3ea 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
@@ -32,7 +32,7 @@ class VertexRDDSuite extends SparkFunSuite with LocalSparkContext {
val n = 100
val verts = vertices(sc, n)
val evens = verts.filter(q => ((q._2 % 2) == 0))
- assert(evens.count === (0 to n).filter(_ % 2 == 0).size)
+ assert(evens.count === (0 to n).count(_ % 2 == 0))
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
index 18be5c0701..3b4209bbc4 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/MinMaxScaler.scala
@@ -166,7 +166,7 @@ class MinMaxScalerModel private[ml] (
// 0 in sparse vector will probably be rescaled to non-zero
val values = vector.toArray
- val size = values.size
+ val size = values.length
var i = 0
while (i < size) {
val raw = if (originalRange(i) != 0) (values(i) - minArray(i)) / originalRange(i) else 0.5
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala
index 769f4406e2..d75b3ef420 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala
@@ -166,7 +166,7 @@ object QuantileDiscretizer extends DefaultParamsReadable[QuantileDiscretizer] wi
* needed, and adding a default split value of 0 if no good candidates are found.
*/
private[feature] def getSplits(candidates: Array[Double]): Array[Double] = {
- val effectiveValues = if (candidates.size != 0) {
+ val effectiveValues = if (candidates.nonEmpty) {
if (candidates.head == Double.NegativeInfinity
&& candidates.last == Double.PositiveInfinity) {
candidates.drop(1).dropRight(1)
@@ -181,7 +181,7 @@ object QuantileDiscretizer extends DefaultParamsReadable[QuantileDiscretizer] wi
candidates
}
- if (effectiveValues.size == 0) {
+ if (effectiveValues.isEmpty) {
Array(Double.NegativeInfinity, 0, Double.PositiveInfinity)
} else {
Array(Double.NegativeInfinity) ++ effectiveValues ++ Array(Double.PositiveInfinity)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index ca0ed95a48..cf17689a3a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -1297,7 +1297,7 @@ private[spark] object SerDe extends Serializable {
def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = {
val m: DenseMatrix = obj.asInstanceOf[DenseMatrix]
- val bytes = new Array[Byte](8 * m.values.size)
+ val bytes = new Array[Byte](8 * m.values.length)
val order = ByteOrder.nativeOrder()
val isTransposed = if (m.isTransposed) 1 else 0
ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values)
@@ -1389,7 +1389,7 @@ private[spark] object SerDe extends Serializable {
def saveState(obj: Object, out: OutputStream, pickler: Pickler): Unit = {
val v: SparseVector = obj.asInstanceOf[SparseVector]
- val n = v.indices.size
+ val n = v.indices.length
val indiceBytes = new Array[Byte](4 * n)
val order = ByteOrder.nativeOrder()
ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala
index 2910c027ae..4308ae04ee 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/impl/GLMClassificationModel.scala
@@ -77,7 +77,7 @@ private[classification] object GLMClassificationModel {
val sqlContext = SQLContext.getOrCreate(sc)
val dataRDD = sqlContext.read.parquet(datapath)
val dataArray = dataRDD.select("weights", "intercept", "threshold").take(1)
- assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath")
+ assert(dataArray.length == 1, s"Unable to load $modelClass data from: $datapath")
val data = dataArray(0)
assert(data.size == 3, s"Unable to load $modelClass data from: $datapath")
val (weights, intercept) = data match {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala
index 54bf5102cc..f0b9d64fc4 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala
@@ -19,6 +19,7 @@ package org.apache.spark.mllib.clustering
import java.util.Random
+import scala.annotation.tailrec
import scala.collection.mutable
import org.apache.spark.Logging
@@ -467,6 +468,7 @@ private[clustering] class ClusteringTreeNode private[clustering] (
* @param cost the cost to the current center
* @return (predicted leaf cluster index, cost)
*/
+ @tailrec
private def predict(pointWithNorm: VectorWithNorm, cost: Double): (Int, Double) = {
if (isLeaf) {
(index, cost)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
index 3b91fe8643..439e4f8672 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
@@ -144,7 +144,7 @@ object KMeansModel extends Loader[KMeansModel] {
val centroids = sqlContext.read.parquet(Loader.dataPath(path))
Loader.checkSchema[Cluster](centroids.schema)
val localCentroids = centroids.rdd.map(Cluster.apply).collect()
- assert(k == localCentroids.size)
+ assert(k == localCentroids.length)
new KMeansModel(localCentroids.sortBy(_.id).map(_.point))
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala
index 3029b15f58..5dde2bdb17 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala
@@ -66,7 +66,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl
*/
@Since("1.1.0")
def confusionMatrix: Matrix = {
- val n = labels.size
+ val n = labels.length
val values = Array.ofDim[Double](n * n)
var i = 0
while (i < n) {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala
index daf6ff4db4..95b2fef27a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala
@@ -58,8 +58,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val accuracy: Double = predictionAndLabels.map { case (predictions, labels) =>
- labels.intersect(predictions).size.toDouble /
- (labels.size + predictions.size - labels.intersect(predictions).size)}.sum / numDocs
+ labels.intersect(predictions).length.toDouble /
+ (labels.length + predictions.length - labels.intersect(predictions).length)}.sum / numDocs
/**
@@ -67,7 +67,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val hammingLoss: Double = predictionAndLabels.map { case (predictions, labels) =>
- labels.size + predictions.size - 2 * labels.intersect(predictions).size
+ labels.length + predictions.length - 2 * labels.intersect(predictions).length
}.sum / (numDocs * numLabels)
/**
@@ -75,8 +75,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val precision: Double = predictionAndLabels.map { case (predictions, labels) =>
- if (predictions.size > 0) {
- predictions.intersect(labels).size.toDouble / predictions.size
+ if (predictions.length > 0) {
+ predictions.intersect(labels).length.toDouble / predictions.length
} else {
0
}
@@ -87,7 +87,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val recall: Double = predictionAndLabels.map { case (predictions, labels) =>
- labels.intersect(predictions).size.toDouble / labels.size
+ labels.intersect(predictions).length.toDouble / labels.length
}.sum / numDocs
/**
@@ -95,7 +95,7 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double]
*/
@Since("1.2.0")
lazy val f1Measure: Double = predictionAndLabels.map { case (predictions, labels) =>
- 2.0 * predictions.intersect(labels).size / (predictions.size + labels.size)
+ 2.0 * predictions.intersect(labels).length / (predictions.length + labels.length)
}.sum / numDocs
private lazy val tpPerClass = predictionAndLabels.flatMap { case (predictions, labels) =>
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
index cffa9fba05..9457c6e9e3 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala
@@ -88,7 +88,7 @@ private object IDF {
}
doc match {
case SparseVector(size, indices, values) =>
- val nnz = indices.size
+ val nnz = indices.length
var k = 0
while (k < nnz) {
if (values(k) > 0) {
@@ -97,7 +97,7 @@ private object IDF {
k += 1
}
case DenseVector(values) =>
- val n = values.size
+ val n = values.length
var j = 0
while (j < n) {
if (values(j) > 0.0) {
@@ -211,7 +211,7 @@ private object IDFModel {
val n = v.size
v match {
case SparseVector(size, indices, values) =>
- val nnz = indices.size
+ val nnz = indices.length
val newValues = new Array[Double](nnz)
var k = 0
while (k < nnz) {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
index af0c8e1d8a..99fcb36f27 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
@@ -55,7 +55,7 @@ class Normalizer @Since("1.1.0") (p: Double) extends VectorTransformer {
vector match {
case DenseVector(vs) =>
val values = vs.clone()
- val size = values.size
+ val size = values.length
var i = 0
while (i < size) {
values(i) /= norm
@@ -64,7 +64,7 @@ class Normalizer @Since("1.1.0") (p: Double) extends VectorTransformer {
Vectors.dense(values)
case SparseVector(size, ids, vs) =>
val values = vs.clone()
- val nnz = values.size
+ val nnz = values.length
var i = 0
while (i < nnz) {
values(i) /= norm
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
index 6fe573c528..500187ae86 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
@@ -132,7 +132,7 @@ class StandardScalerModel @Since("1.3.0") (
vector match {
case DenseVector(vs) =>
val values = vs.clone()
- val size = values.size
+ val size = values.length
if (withStd) {
var i = 0
while (i < size) {
@@ -153,7 +153,7 @@ class StandardScalerModel @Since("1.3.0") (
vector match {
case DenseVector(vs) =>
val values = vs.clone()
- val size = values.size
+ val size = values.length
var i = 0
while(i < size) {
values(i) *= (if (std(i) != 0.0) 1.0 / std(i) else 0.0)
@@ -164,7 +164,7 @@ class StandardScalerModel @Since("1.3.0") (
// For sparse vector, the `index` array inside sparse vector object will not be changed,
// so we can re-use it to save memory.
val values = vs.clone()
- val nnz = values.size
+ val nnz = values.length
var i = 0
while (i < nnz) {
values(i) *= (if (std(indices(i)) != 0.0) 1.0 / std(indices(i)) else 0.0)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
index 3241ebeb22..b046f684ca 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala
@@ -346,9 +346,9 @@ class Word2Vec extends Serializable with Logging {
if (alpha < learningRate * 0.0001) alpha = learningRate * 0.0001
logInfo("wordCount = " + wordCount + ", alpha = " + alpha)
}
- wc += sentence.size
+ wc += sentence.length
var pos = 0
- while (pos < sentence.size) {
+ while (pos < sentence.length) {
val word = sentence(pos)
val b = random.nextInt(window)
// Train Skip-gram
@@ -356,7 +356,7 @@ class Word2Vec extends Serializable with Logging {
while (a < window * 2 + 1 - b) {
if (a != window) {
val c = pos - window + a
- if (c >= 0 && c < sentence.size) {
+ if (c >= 0 && c < sentence.length) {
val lastWord = sentence(c)
val l1 = lastWord * vectorSize
val neu1e = new Array[Float](vectorSize)
@@ -579,7 +579,7 @@ object Word2VecModel extends Loader[Word2VecModel] {
private def buildWordVectors(model: Map[String, Array[Float]]): Array[Float] = {
require(model.nonEmpty, "Word2VecMap should be non-empty")
- val (vectorSize, numWords) = (model.head._2.size, model.size)
+ val (vectorSize, numWords) = (model.head._2.length, model.size)
val wordList = model.keys.toArray
val wordVectors = new Array[Float](vectorSize * numWords)
var i = 0
@@ -615,7 +615,7 @@ object Word2VecModel extends Loader[Word2VecModel] {
val sqlContext = SQLContext.getOrCreate(sc)
import sqlContext.implicits._
- val vectorSize = model.values.head.size
+ val vectorSize = model.values.head.length
val numWords = model.size
val metadata = compact(render(
("class" -> classNameV1_0) ~ ("version" -> formatVersionV1_0) ~
@@ -646,7 +646,7 @@ object Word2VecModel extends Loader[Word2VecModel] {
(loadedClassName, loadedVersion) match {
case (classNameV1_0, "1.0") =>
val model = SaveLoadV1_0.load(sc, path)
- val vectorSize = model.getVectors.values.head.size
+ val vectorSize = model.getVectors.values.head.length
val numWords = model.getVectors.size
require(expectedVectorSize == vectorSize,
s"Word2VecModel requires each word to be mapped to a vector of size " +
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
index b35d7217d6..f5b4f2535f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
@@ -232,7 +232,7 @@ class FPGrowth private (
partitioner: Partitioner): Array[Item] = {
data.flatMap { t =>
val uniq = t.toSet
- if (t.size != uniq.size) {
+ if (t.length != uniq.size) {
throw new SparkException(s"Items in a transaction must be unique but got ${t.toSeq}.")
}
t
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
index df9f4ae145..d2687dc11b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala
@@ -75,7 +75,7 @@ private[spark] object BLAS extends Serializable with Logging {
val xValues = x.values
val xIndices = x.indices
val yValues = y.values
- val nnz = xIndices.size
+ val nnz = xIndices.length
if (a == 1.0) {
var k = 0
@@ -135,7 +135,7 @@ private[spark] object BLAS extends Serializable with Logging {
val xValues = x.values
val xIndices = x.indices
val yValues = y.values
- val nnz = xIndices.size
+ val nnz = xIndices.length
var sum = 0.0
var k = 0
@@ -154,8 +154,8 @@ private[spark] object BLAS extends Serializable with Logging {
val xIndices = x.indices
val yValues = y.values
val yIndices = y.indices
- val nnzx = xIndices.size
- val nnzy = yIndices.size
+ val nnzx = xIndices.length
+ val nnzy = yIndices.length
var kx = 0
var ky = 0
@@ -188,7 +188,7 @@ private[spark] object BLAS extends Serializable with Logging {
val sxIndices = sx.indices
val sxValues = sx.values
val dyValues = dy.values
- val nnz = sxIndices.size
+ val nnz = sxIndices.length
var i = 0
var k = 0
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala
index ffdcddec11..e4494792bb 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/CholeskyDecomposition.scala
@@ -33,7 +33,7 @@ private[spark] object CholeskyDecomposition {
* @return the solution array
*/
def solve(A: Array[Double], bx: Array[Double]): Array[Double] = {
- val k = bx.size
+ val k = bx.length
val info = new intW(0)
lapack.dppsv("U", k, 1, A, bx, k, info)
val code = info.`val`
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 b08da4fb55..0fdb402fd6 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
@@ -987,7 +987,7 @@ object Matrices {
def horzcat(matrices: Array[Matrix]): Matrix = {
if (matrices.isEmpty) {
return new DenseMatrix(0, 0, Array[Double]())
- } else if (matrices.size == 1) {
+ } else if (matrices.length == 1) {
return matrices(0)
}
val numRows = matrices(0).numRows
@@ -1046,7 +1046,7 @@ object Matrices {
def vertcat(matrices: Array[Matrix]): Matrix = {
if (matrices.isEmpty) {
return new DenseMatrix(0, 0, Array[Double]())
- } else if (matrices.size == 1) {
+ } else if (matrices.length == 1) {
return matrices(0)
}
val numCols = matrices(0).numCols
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 09527dcf5d..ae1faf6a2d 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
@@ -176,7 +176,7 @@ class BlockMatrix @Since("1.3.0") (
val numColBlocks = math.ceil(numCols() * 1.0 / colsPerBlock).toInt
private[mllib] def createPartitioner(): GridPartitioner =
- GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = blocks.partitions.size)
+ GridPartitioner(numRowBlocks, numColBlocks, suggestedNumPartitions = blocks.partitions.length)
private lazy val blockInfo = blocks.mapValues(block => (block.numRows, block.numCols)).cache()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
index e8de515211..06b9c4ac67 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
@@ -120,9 +120,9 @@ class IndexedRowMatrix @Since("1.0.0") (
val rowIndex = row.index
row.vector match {
case SparseVector(size, indices, values) =>
- Iterator.tabulate(indices.size)(i => MatrixEntry(rowIndex, indices(i), values(i)))
+ Iterator.tabulate(indices.length)(i => MatrixEntry(rowIndex, indices(i), values(i)))
case DenseVector(values) =>
- Iterator.tabulate(values.size)(i => MatrixEntry(rowIndex, i, values(i)))
+ Iterator.tabulate(values.length)(i => MatrixEntry(rowIndex, i, values(i)))
}
}
new CoordinateMatrix(entries, numRows(), numCols())
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index 3e619c4264..a7a843ab76 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -226,12 +226,12 @@ class ALS private (
val sc = ratings.context
val numUserBlocks = if (this.numUserBlocks == -1) {
- math.max(sc.defaultParallelism, ratings.partitions.size / 2)
+ math.max(sc.defaultParallelism, ratings.partitions.length / 2)
} else {
this.numUserBlocks
}
val numProductBlocks = if (this.numProductBlocks == -1) {
- math.max(sc.defaultParallelism, ratings.partitions.size / 2)
+ math.max(sc.defaultParallelism, ratings.partitions.length / 2)
} else {
this.numProductBlocks
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
index 73da899a0e..f7e3c5c501 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
@@ -350,7 +350,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
val partialWeightsArray = scaler.transform(
Vectors.dense(weightsArray.slice(start, end))).toArray
- System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.size)
+ System.arraycopy(partialWeightsArray, 0, weightsArray, start, partialWeightsArray.length)
i += 1
}
weights = Vectors.dense(weightsArray)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala
index 02af281fb7..a6e1767fe2 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/impl/GLMRegressionModel.scala
@@ -74,7 +74,7 @@ private[regression] object GLMRegressionModel {
val sqlContext = SQLContext.getOrCreate(sc)
val dataRDD = sqlContext.read.parquet(datapath)
val dataArray = dataRDD.select("weights", "intercept").take(1)
- assert(dataArray.size == 1, s"Unable to load $modelClass data from: $datapath")
+ assert(dataArray.length == 1, s"Unable to load $modelClass data from: $datapath")
val data = dataArray(0)
assert(data.size == 2, s"Unable to load $modelClass data from: $datapath")
data match {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
index 40440d50fc..76c32208ea 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
@@ -17,6 +17,7 @@
package org.apache.spark.mllib.tree
+import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.collection.mutable
@@ -286,6 +287,7 @@ object DecisionTree extends Serializable with Logging {
* This index is different from the index used during training a particular
* group of nodes on one call to [[findBestSplits()]].
*/
+ @tailrec
private def predictNodeIndex(
node: Node,
binnedFeatures: Array[Int],
@@ -350,7 +352,7 @@ object DecisionTree extends Serializable with Logging {
featuresForNode: Option[Array[Int]]): Unit = {
val numFeaturesPerNode = if (featuresForNode.nonEmpty) {
// Use subsampled features
- featuresForNode.get.size
+ featuresForNode.get.length
} else {
// Use all features
agg.metadata.numFeatures
@@ -411,7 +413,7 @@ object DecisionTree extends Serializable with Logging {
if (featuresForNode.nonEmpty) {
// Use subsampled features
var featureIndexIdx = 0
- while (featureIndexIdx < featuresForNode.get.size) {
+ while (featureIndexIdx < featuresForNode.get.length) {
val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx))
agg.update(featureIndexIdx, binIndex, label, instanceWeight)
featureIndexIdx += 1
@@ -483,7 +485,7 @@ object DecisionTree extends Serializable with Logging {
*/
// numNodes: Number of nodes in this group
- val numNodes = nodesForGroup.values.map(_.size).sum
+ val numNodes = nodesForGroup.values.map(_.length).sum
logDebug("numNodes = " + numNodes)
logDebug("numFeatures = " + metadata.numFeatures)
logDebug("numClasses = " + metadata.numClasses)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
index a74197278d..09017d482a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
@@ -104,9 +104,9 @@ private[tree] class VarianceAggregator()
*/
private[spark] class VarianceCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) {
- require(stats.size == 3,
+ require(stats.length == 3,
s"VarianceCalculator requires sufficient statistics array stats to be of length 3," +
- s" but was given array of length ${stats.size}.")
+ s" but was given array of length ${stats.length}.")
/**
* Make a deep copy of this [[ImpurityCalculator]].
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
index ec5d7b9189..e007ee1105 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
@@ -250,7 +250,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging {
val nodes = dataRDD.rdd.map(NodeData.apply)
// Build node data into a tree.
val trees = constructTrees(nodes)
- assert(trees.size == 1,
+ assert(trees.length == 1,
"Decision tree should contain exactly one tree but got ${trees.size} trees.")
val model = new DecisionTreeModel(trees(0), Algo.fromString(algo))
assert(model.numNodes == numNodes, s"Unable to load DecisionTreeModel data from: $datapath." +
@@ -266,7 +266,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging {
.map { case (treeId, data) =>
(treeId, constructTree(data))
}.sortBy(_._1)
- val numTrees = trees.size
+ val numTrees = trees.length
val treeIndices = trees.map(_._1).toSeq
assert(treeIndices == (0 until numTrees),
s"Tree indices must start from 0 and increment by 1, but we found $treeIndices.")
diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
index eeeabfe359..916fff14a7 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
@@ -95,7 +95,9 @@ public class JavaFPGrowthSuite implements Serializable {
try {
model.save(sc.sc(), outputPath);
- FPGrowthModel newModel = FPGrowthModel.load(sc.sc(), outputPath);
+ @SuppressWarnings("unchecked")
+ FPGrowthModel<String> newModel =
+ (FPGrowthModel<String>) FPGrowthModel.load(sc.sc(), outputPath);
List<FPGrowth.FreqItemset<String>> freqItemsets = newModel.freqItemsets().toJavaRDD()
.collect();
assertEquals(18, freqItemsets.size());
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
index 8fb8886645..a200e9454b 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -38,7 +38,7 @@ class RidgeRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]): Double = {
predictions.zip(input).map { case (prediction, expected) =>
(prediction - expected.label) * (prediction - expected.label)
- }.reduceLeft(_ + _) / predictions.size
+ }.sum / predictions.size
}
test("ridge regression can help avoid overfitting") {
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala
index 50441816ec..0921fdba33 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/StreamingTestSuite.scala
@@ -164,7 +164,7 @@ class StreamingTestSuite extends SparkFunSuite with TestSuiteBase {
// number of batches seen so far does not exceed testWindow, expect counts to continue growing
for (i <- 0 until testWindow) {
- assert(outputCounts.drop(2 * i).take(2).forall(_ == (i + 1) * pointsPerBatch / 2))
+ assert(outputCounts.slice(2 * i, 2 * i + 2).forall(_ == (i + 1) * pointsPerBatch / 2))
}
// number of batches seen exceeds testWindow, expect counts to be constant
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 36eb59ef5e..fbbc3ee891 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis
import java.lang.reflect.Modifier
+import scala.annotation.tailrec
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.sql.AnalysisException
@@ -689,6 +690,7 @@ class Analyzer(
* Resolve the expression on a specified logical plan and it's child (recursively), until
* the expression is resolved or meet a non-unary node or Subquery.
*/
+ @tailrec
private def resolveExpressionRecursively(expr: Expression, plan: LogicalPlan): Expression = {
val resolved = resolveExpression(expr, plan)
if (resolved.resolved) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 1072158f04..a965cc8d53 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -925,7 +925,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression {
(c, evPrim, evNull) =>
s"""
- final $rowClass $result = new $rowClass(${fieldsCasts.size});
+ final $rowClass $result = new $rowClass(${fieldsCasts.length});
final InternalRow $tmpRow = $c;
$fieldsEvalCode
$evPrim = $result.copy();
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
index 4cb6af9d9f..cf73e36d22 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.expressions.codegen
+import scala.annotation.tailrec
+
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData}
@@ -120,6 +122,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection]
ExprCode(code, "false", output)
}
+ @tailrec
private def convertToSafe(
ctx: CodegenContext,
input: String,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 0df8101d94..87e43429e6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -159,7 +159,7 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression {
TypeCheckResult.TypeCheckFailure(
s"Only foldable StringType expressions are allowed to appear at odd position , got :" +
s" ${invalidNames.mkString(",")}")
- } else if (names.forall(_ != null)){
+ } else if (!names.contains(null)){
TypeCheckResult.TypeCheckSuccess
} else {
TypeCheckResult.TypeCheckFailure("Field name should not be null")
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
index 33bd3f2095..8f260ad151 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/misc.scala
@@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.expressions
import java.security.{MessageDigest, NoSuchAlgorithmException}
import java.util.zip.CRC32
+import scala.annotation.tailrec
+
import org.apache.commons.codec.digest.DigestUtils
import org.apache.spark.sql.catalyst.InternalRow
@@ -352,6 +354,7 @@ case class Murmur3Hash(children: Seq[Expression], seed: Int) extends Expression
}
}
+ @tailrec
private def computeHash(
input: String,
dataType: DataType,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
index 737346dc79..75ecbaa453 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.expressions
+import scala.annotation.tailrec
import scala.language.existentials
import scala.reflect.ClassTag
@@ -370,6 +371,7 @@ case class MapObjects private(
lambdaFunction: Expression,
inputData: Expression) extends Expression with NonSQLExpression {
+ @tailrec
private def itemAccessorMethod(dataType: DataType): String => String = dataType match {
case NullType =>
val nullTypeClassName = NullType.getClass.getName + ".MODULE$"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index 059d8ff87b..c83ec0fcb5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.catalyst.optimizer
+import scala.annotation.tailrec
import scala.collection.immutable.HashSet
import org.apache.spark.sql.catalyst.analysis.{CleanupAliases, EliminateSubqueryAliases}
@@ -915,6 +916,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper {
* @param input a list of LogicalPlans to join.
* @param conditions a list of condition for join.
*/
+ @tailrec
def createOrderedJoin(input: Seq[LogicalPlan], conditions: Seq[Expression]): LogicalPlan = {
assert(input.size >= 2)
if (input.size == 2) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
index 9ff41f5bec..7f96db137e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
@@ -16,6 +16,8 @@
*/
package org.apache.spark.sql.catalyst.parser
+import scala.annotation.tailrec
+
import org.antlr.runtime._
import org.antlr.runtime.tree.CommonTree
@@ -71,6 +73,7 @@ object ParseDriver extends Logging {
logInfo(s"Parse completed.")
// Find the non null token tree in the result.
+ @tailrec
def nonNullToken(tree: CommonTree): CommonTree = {
if (tree.token != null || tree.getChildCount == 0) tree
else nonNullToken(tree.getChild(0).asInstanceOf[CommonTree])
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index f184d72285..5393cb8ab3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -22,6 +22,8 @@ import java.text.{DateFormat, SimpleDateFormat}
import java.util.{Calendar, TimeZone}
import javax.xml.bind.DatatypeConverter
+import scala.annotation.tailrec
+
import org.apache.spark.unsafe.types.UTF8String
/**
@@ -117,6 +119,7 @@ object DateTimeUtils {
}
}
+ @tailrec
def stringToTime(s: String): java.util.Date = {
val indexOfGMT = s.indexOf("GMT")
if (indexOfGMT != -1) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
index 43f707f444..d9a9b6151a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
@@ -104,12 +104,12 @@ package object util {
}
def sideBySide(left: Seq[String], right: Seq[String]): Seq[String] = {
- val maxLeftSize = left.map(_.size).max
+ val maxLeftSize = left.map(_.length).max
val leftPadded = left ++ Seq.fill(math.max(right.size - left.size, 0))("")
val rightPadded = right ++ Seq.fill(math.max(left.size - right.size, 0))("")
leftPadded.zip(rightPadded).map {
- case (l, r) => (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.size) + 3)) + r
+ case (l, r) => (if (l == r) " " else "!") + l + (" " * ((maxLeftSize - l.length) + 3)) + r
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
index 5ff5435d5a..271ca95a24 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala
@@ -292,7 +292,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru
builder.append("struct<")
builder.append(fieldTypes.mkString(", "))
if (fields.length > 2) {
- if (fields.length - fieldTypes.size == 1) {
+ if (fields.length - fieldTypes.length == 1) {
builder.append(" ... 1 more field")
} else {
builder.append(" ... " + (fields.length - 2) + " more fields")
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala
index 9fba7924e9..3c2f8a2887 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala
@@ -40,7 +40,7 @@ class RandomDataGeneratorSuite extends SparkFunSuite {
if (nullable) {
assert(Iterator.fill(100)(generator()).contains(null))
} else {
- assert(Iterator.fill(100)(generator()).forall(_ != null))
+ assert(!Iterator.fill(100)(generator()).contains(null))
}
for (_ <- 1 to 10) {
val generatedValue = generator()
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
index 43a3eb9dec..5d688e2fe4 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
@@ -97,7 +97,7 @@ class SubexpressionEliminationSuite extends SparkFunSuite {
equivalence.addExprTree(add2, true)
// Should only have one equivalence for `one + two`
- assert(equivalence.getAllEquivalentExprs.filter(_.size > 1).size == 1)
+ assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 1)
assert(equivalence.getAllEquivalentExprs.filter(_.size > 1).head.size == 4)
// Set up the expressions
@@ -116,7 +116,7 @@ class SubexpressionEliminationSuite extends SparkFunSuite {
equivalence.addExprTree(sum, true)
// (one * two), (one * two) * (one * two) and sqrt( (one * two) * (one * two) ) should be found
- assert(equivalence.getAllEquivalentExprs.filter(_.size > 1).size == 3)
+ assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 3)
assert(equivalence.getEquivalentExprs(mul).size == 3)
assert(equivalence.getEquivalentExprs(mul2).size == 3)
assert(equivalence.getEquivalentExprs(sqrt).size == 2)
@@ -144,7 +144,7 @@ class SubexpressionEliminationSuite extends SparkFunSuite {
equivalence.addExprTree(price, false)
equivalence.addExprTree(discount, false)
// quantity, price, discount and (price * (1 - discount))
- assert(equivalence.getAllEquivalentExprs.filter(_.size > 1).size == 4)
+ assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 4)
}
test("Expression equivalence - non deterministic") {
@@ -164,7 +164,7 @@ class SubexpressionEliminationSuite extends SparkFunSuite {
var equivalence = new EquivalentExpressions
equivalence.addExprTree(add, true)
// the `two` inside `explode` should not be added
- assert(equivalence.getAllEquivalentExprs.filter(_.size > 1).size == 0)
- assert(equivalence.getAllEquivalentExprs.filter(_.size == 1).size == 3) // add, two, explode
+ assert(equivalence.getAllEquivalentExprs.count(_.size > 1) == 0)
+ assert(equivalence.getAllEquivalentExprs.count(_.size == 1) == 3) // add, two, explode
}
}
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java
index 8a0d7f8b12..2a78058838 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java
@@ -251,7 +251,6 @@ public final class ColumnarBatch {
@Override
public Row next() {
- assert(hasNext());
while (rowId < maxRows && ColumnarBatch.this.filteredRows[rowId]) {
++rowId;
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala
index 13142d0e61..0a156ea99a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/ContinuousQueryManager.scala
@@ -55,9 +55,8 @@ class ContinuousQueryManager(sqlContext: SQLContext) {
* @since 2.0.0
*/
def get(name: String): ContinuousQuery = activeQueriesLock.synchronized {
- activeQueries.get(name).getOrElse {
- throw new IllegalArgumentException(s"There is no active query with name $name")
- }
+ activeQueries.getOrElse(name,
+ throw new IllegalArgumentException(s"There is no active query with name $name"))
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala
index 68a251757c..d8af799d89 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala
@@ -94,7 +94,7 @@ private[r] object SQLUtils {
}
def createDF(rdd: RDD[Array[Byte]], schema: StructType, sqlContext: SQLContext): DataFrame = {
- val num = schema.fields.size
+ val num = schema.fields.length
val rowRDD = rdd.map(bytesToRow(_, schema))
sqlContext.createDataFrame(rowRDD, schema)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
index fee36f6023..78664baa56 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnAccessor.scala
@@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.columnar
import java.nio.{ByteBuffer, ByteOrder}
+import scala.annotation.tailrec
+
import org.apache.spark.sql.catalyst.expressions.{MutableRow, UnsafeArrayData, UnsafeMapData, UnsafeRow}
import org.apache.spark.sql.execution.columnar.compression.CompressibleColumnAccessor
import org.apache.spark.sql.types._
@@ -120,6 +122,7 @@ private[columnar] class MapColumnAccessor(buffer: ByteBuffer, dataType: MapType)
with NullableColumnAccessor
private[columnar] object ColumnAccessor {
+ @tailrec
def apply(dataType: DataType, buffer: ByteBuffer): ColumnAccessor = {
val buf = buffer.order(ByteOrder.nativeOrder)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
index 9c908b2877..3ec01185c4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.columnar
import java.math.{BigDecimal, BigInteger}
import java.nio.ByteBuffer
+import scala.annotation.tailrec
import scala.reflect.runtime.universe.TypeTag
import org.apache.spark.sql.catalyst.InternalRow
@@ -548,7 +549,7 @@ private[columnar] object LARGE_DECIMAL {
private[columnar] case class STRUCT(dataType: StructType)
extends ColumnType[UnsafeRow] with DirectCopyColumnType[UnsafeRow] {
- private val numOfFields: Int = dataType.fields.size
+ private val numOfFields: Int = dataType.fields.length
override def defaultSize: Int = 20
@@ -663,6 +664,7 @@ private[columnar] case class MAP(dataType: MapType)
}
private[columnar] object ColumnType {
+ @tailrec
def apply(dataType: DataType): ColumnType[_] = {
dataType match {
case NullType => NULL
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala
index 22d4278085..1f964b1fc1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarTableScan.scala
@@ -147,7 +147,7 @@ private[sql] case class InMemoryRelation(
// may result malformed rows, causing ArrayIndexOutOfBoundsException, which is somewhat
// hard to decipher.
assert(
- row.numFields == columnBuilders.size,
+ row.numFields == columnBuilders.length,
s"Row column number mismatch, expected ${output.size} columns, " +
s"but got ${row.numFields}." +
s"\nRow content: $row")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
index e9afee1cc5..d2d7996f56 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
@@ -204,16 +204,16 @@ object CSVRelation extends Logging {
val rowArray = new Array[Any](safeRequiredIndices.length)
val requiredSize = requiredFields.length
tokenizedRDD.flatMap { tokens =>
- if (params.dropMalformed && schemaFields.length != tokens.size) {
+ if (params.dropMalformed && schemaFields.length != tokens.length) {
logWarning(s"Dropping malformed line: ${tokens.mkString(params.delimiter.toString)}")
None
- } else if (params.failFast && schemaFields.length != tokens.size) {
+ } else if (params.failFast && schemaFields.length != tokens.length) {
throw new RuntimeException(s"Malformed line in FAILFAST mode: " +
s"${tokens.mkString(params.delimiter.toString)}")
} else {
- val indexSafeTokens = if (params.permissive && schemaFields.length > tokens.size) {
- tokens ++ new Array[String](schemaFields.length - tokens.size)
- } else if (params.permissive && schemaFields.length < tokens.size) {
+ val indexSafeTokens = if (params.permissive && schemaFields.length > tokens.length) {
+ tokens ++ new Array[String](schemaFields.length - tokens.length)
+ } else if (params.permissive && schemaFields.length < tokens.length) {
tokens.take(schemaFields.length)
} else {
tokens
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
index ed02b3f95f..4dd3c50cdf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala
@@ -212,14 +212,14 @@ private[sql] object JDBCRDD extends Logging {
// We can't compile Or filter unless both sub-filters are compiled successfully.
// It applies too for the following And filter.
// If we can make sure compileFilter supports all filters, we can remove this check.
- val or = Seq(f1, f2).map(compileFilter(_)).flatten
+ val or = Seq(f1, f2).flatMap(compileFilter(_))
if (or.size == 2) {
or.map(p => s"($p)").mkString(" OR ")
} else {
null
}
case And(f1, f2) =>
- val and = Seq(f1, f2).map(compileFilter(_)).flatten
+ val and = Seq(f1, f2).flatMap(compileFilter(_))
if (and.size == 2) {
and.map(p => s"($p)").mkString(" AND ")
} else {
@@ -304,7 +304,7 @@ private[sql] class JDBCRDD(
* `filters`, but as a WHERE clause suitable for injection into a SQL query.
*/
private val filterWhereClause: String =
- filters.map(JDBCRDD.compileFilter).flatten.mkString(" AND ")
+ filters.flatMap(JDBCRDD.compileFilter).mkString(" AND ")
/**
* A WHERE clause representing both `filters`, if any, and the current partition.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala
index 835e7ba6c5..f9d10292f1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala
@@ -335,7 +335,7 @@ private[spark] class SQLHistoryListener(conf: SparkConf, sparkUI: SparkUI)
taskEnd.taskInfo.accumulables.flatMap { a =>
// Filter out accumulators that are not SQL metrics
// For now we assume all SQL metrics are Long's that have been JSON serialized as String's
- if (a.metadata.exists(_ == SQLMetrics.ACCUM_IDENTIFIER)) {
+ if (a.metadata.contains(SQLMetrics.ACCUM_IDENTIFIER)) {
val newValue = new LongSQLMetricValue(a.update.map(_.toString.toLong).getOrElse(0L))
Some(a.copy(update = Some(newValue)))
} else {
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
index 0d4c128cb3..ee85626435 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java
@@ -355,27 +355,27 @@ public class JavaDataFrameSuite {
DataFrame df = context.range(1000);
BloomFilter filter1 = df.stat().bloomFilter("id", 1000, 0.03);
- assert (filter1.expectedFpp() - 0.03 < 1e-3);
+ Assert.assertTrue(filter1.expectedFpp() - 0.03 < 1e-3);
for (int i = 0; i < 1000; i++) {
- assert (filter1.mightContain(i));
+ Assert.assertTrue(filter1.mightContain(i));
}
BloomFilter filter2 = df.stat().bloomFilter(col("id").multiply(3), 1000, 0.03);
- assert (filter2.expectedFpp() - 0.03 < 1e-3);
+ Assert.assertTrue(filter2.expectedFpp() - 0.03 < 1e-3);
for (int i = 0; i < 1000; i++) {
- assert (filter2.mightContain(i * 3));
+ Assert.assertTrue(filter2.mightContain(i * 3));
}
BloomFilter filter3 = df.stat().bloomFilter("id", 1000, 64 * 5);
- assert (filter3.bitSize() == 64 * 5);
+ Assert.assertTrue(filter3.bitSize() == 64 * 5);
for (int i = 0; i < 1000; i++) {
- assert (filter3.mightContain(i));
+ Assert.assertTrue(filter3.mightContain(i));
}
BloomFilter filter4 = df.stat().bloomFilter(col("id").multiply(3), 1000, 64 * 5);
- assert (filter4.bitSize() == 64 * 5);
+ Assert.assertTrue(filter4.bitSize() == 64 * 5);
for (int i = 0; i < 1000; i++) {
- assert (filter4.mightContain(i * 3));
+ Assert.assertTrue(filter4.mightContain(i * 3));
}
}
}
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
index 1181244c8a..e0e56f3fbf 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java
@@ -304,15 +304,12 @@ public class JavaDatasetSuite implements Serializable {
Assert.assertEquals(Arrays.asList("abc", "abc"), subtracted.collectAsList());
}
- private <T> Set<T> toSet(List<T> records) {
- Set<T> set = new HashSet<T>();
- for (T record : records) {
- set.add(record);
- }
- return set;
+ private static <T> Set<T> toSet(List<T> records) {
+ return new HashSet<>(records);
}
- private <T> Set<T> asSet(T... records) {
+ @SafeVarargs
+ private static <T> Set<T> asSet(T... records) {
return toSet(Arrays.asList(records));
}
@@ -529,7 +526,7 @@ public class JavaDatasetSuite implements Serializable {
Encoders.kryo(PrivateClassTest.class);
}
- public class SimpleJavaBean implements Serializable {
+ public static class SimpleJavaBean implements Serializable {
private boolean a;
private int b;
private byte[] c;
@@ -612,7 +609,7 @@ public class JavaDatasetSuite implements Serializable {
}
}
- public class SimpleJavaBean2 implements Serializable {
+ public static class SimpleJavaBean2 implements Serializable {
private Timestamp a;
private Date b;
private java.math.BigDecimal c;
@@ -650,7 +647,7 @@ public class JavaDatasetSuite implements Serializable {
}
}
- public class NestedJavaBean implements Serializable {
+ public static class NestedJavaBean implements Serializable {
private SimpleJavaBean a;
public SimpleJavaBean getA() {
@@ -745,7 +742,7 @@ public class JavaDatasetSuite implements Serializable {
ds.collect();
}
- public class SmallBean implements Serializable {
+ public static class SmallBean implements Serializable {
private String a;
private int b;
@@ -780,7 +777,7 @@ public class JavaDatasetSuite implements Serializable {
}
}
- public class NestedSmallBean implements Serializable {
+ public static class NestedSmallBean implements Serializable {
private SmallBean f;
public SmallBean getF() {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index 84f30c0aaf..a824759cb8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -603,7 +603,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext {
assert(parquetDF.inputFiles.nonEmpty)
val unioned = jsonDF.unionAll(parquetDF).inputFiles.sorted
- val allFiles = (jsonDF.inputFiles ++ parquetDF.inputFiles).toSet.toArray.sorted
+ val allFiles = (jsonDF.inputFiles ++ parquetDF.inputFiles).distinct.sorted
assert(unioned === allFiles)
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 16e769feca..f59faa0dc2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -1562,16 +1562,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
e.message.contains("Cannot save interval data type into external storage")
})
- def checkIntervalParseError(s: String): Unit = {
- val e = intercept[AnalysisException] {
- sql(s)
- }
- e.message.contains("at least one time unit should be given for interval literal")
+ val e1 = intercept[AnalysisException] {
+ sql("select interval")
}
-
- checkIntervalParseError("select interval")
+ assert(e1.message.contains("at least one time unit should be given for interval literal"))
// Currently we don't yet support nanosecond
- checkIntervalParseError("select interval 23 nanosecond")
+ val e2 = intercept[AnalysisException] {
+ sql("select interval 23 nanosecond")
+ }
+ assert(e2.message.contains("cannot recognize input near"))
}
test("SPARK-8945: add and subtract expressions for interval type") {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
index 5d84feb483..3e91569109 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala
@@ -385,7 +385,7 @@ private[hive] trait HiveInspectors {
(o: Any) =>
if (o != null) {
val s = o.asInstanceOf[UTF8String].toString
- new HiveVarchar(s, s.size)
+ new HiveVarchar(s, s.length)
} else {
null
}
@@ -394,7 +394,7 @@ private[hive] trait HiveInspectors {
(o: Any) =>
if (o != null) {
val s = o.asInstanceOf[UTF8String].toString
- new HiveChar(s, s.size)
+ new HiveChar(s, s.length)
} else {
null
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 3788736fd1..ee8ec2d9f7 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -149,11 +149,10 @@ private[hive] class HiveMetastoreCatalog(val client: HiveClient, hive: HiveConte
def getColumnNames(colType: String): Seq[String] = {
table.properties.get(s"spark.sql.sources.schema.num${colType.capitalize}Cols").map {
numCols => (0 until numCols.toInt).map { index =>
- table.properties.get(s"spark.sql.sources.schema.${colType}Col.$index").getOrElse {
+ table.properties.getOrElse(s"spark.sql.sources.schema.${colType}Col.$index",
throw new AnalysisException(
s"Could not read $colType columns from the metastore because it is corrupted " +
- s"(missing part $index of it, $numCols parts are expected).")
- }
+ s"(missing part $index of it, $numCols parts are expected)."))
}
}.getOrElse(Nil)
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
index 145b5f7cc2..a89ed48608 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala
@@ -178,7 +178,7 @@ case class InsertIntoHiveTable(
// loadPartition call orders directories created on the iteration order of the this map
val orderedPartitionSpec = new util.LinkedHashMap[String, String]()
table.hiveQlTable.getPartCols.asScala.foreach { entry =>
- orderedPartitionSpec.put(entry.getName, partitionSpec.get(entry.getName).getOrElse(""))
+ orderedPartitionSpec.put(entry.getName, partitionSpec.getOrElse(entry.getName, ""))
}
// inheritTableSpecs is set to true. It should be set to false for a IMPORT query
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
index eedb42c061..d397688245 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
@@ -160,7 +160,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
require(batchDuration != null, "Batch duration has not been set")
// assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration +
// " is very low")
- require(getOutputStreams().size > 0, "No output operations registered, so nothing to execute")
+ require(getOutputStreams().nonEmpty, "No output operations registered, so nothing to execute")
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
index a25dada5ea..7fba2e8ec0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
@@ -276,7 +276,7 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]](
config)
case None => context.sparkContext.newAPIHadoopFile[K, V, F](file)
}
- if (rdd.partitions.size == 0) {
+ if (rdd.partitions.isEmpty) {
logError("File " + file + " has no data in it. Spark Streaming can only ingest " +
"files that have been \"moved\" to the directory assigned to the file stream. " +
"Refer to the streaming programming guide for more details.")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
index 49d8f14f4c..fd3e72e41b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala
@@ -108,7 +108,7 @@ abstract class ReceiverInputDStream[T: ClassTag](_ssc: StreamingContext)
} else {
// Else, create a BlockRDD. However, if there are some blocks with WAL info but not
// others then that is unexpected and log a warning accordingly.
- if (blockInfos.find(_.walRecordHandleOption.nonEmpty).nonEmpty) {
+ if (blockInfos.exists(_.walRecordHandleOption.nonEmpty)) {
if (WriteAheadLogUtils.enableReceiverLog(ssc.conf)) {
logError("Some blocks do not have Write Ahead Log information; " +
"this is unexpected and data may not be recoverable after driver failures")
@@ -119,7 +119,7 @@ abstract class ReceiverInputDStream[T: ClassTag](_ssc: StreamingContext)
val validBlockIds = blockIds.filter { id =>
ssc.sparkContext.env.blockManager.master.contains(id)
}
- if (validBlockIds.size != blockIds.size) {
+ if (validBlockIds.length != blockIds.length) {
logWarning("Some blocks could not be recovered as they were not found in memory. " +
"To prevent such data loss, enable Write Ahead Log (see programming guide " +
"for more details.")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
index 5359549085..817ecc1ace 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala
@@ -129,7 +129,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag](
val numNewValues = newRDDs.size
val mergeValues = (arrayOfValues: Array[Iterable[V]]) => {
- if (arrayOfValues.size != 1 + numOldValues + numNewValues) {
+ if (arrayOfValues.length != 1 + numOldValues + numNewValues) {
throw new Exception("Unexpected number of sequences of reduced values")
}
// Getting reduced values "old time steps" that will be removed from current window
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
index 2b07dd6185..c1846a31f6 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/UnionDStream.scala
@@ -29,8 +29,8 @@ class UnionDStream[T: ClassTag](parents: Array[DStream[T]])
extends DStream[T](parents.head.ssc) {
require(parents.length > 0, "List of DStreams to union is empty")
- require(parents.map(_.ssc).distinct.size == 1, "Some of the DStreams have different contexts")
- require(parents.map(_.slideDuration).distinct.size == 1,
+ require(parents.map(_.ssc).distinct.length == 1, "Some of the DStreams have different contexts")
+ require(parents.map(_.slideDuration).distinct.length == 1,
"Some of the DStreams have different slide durations")
override def dependencies: List[DStream[_]] = parents.toList
@@ -44,7 +44,7 @@ class UnionDStream[T: ClassTag](parents: Array[DStream[T]])
case None => throw new SparkException("Could not generate RDD from a parent for unifying at" +
s" time $validTime")
}
- if (rdds.size > 0) {
+ if (rdds.nonEmpty) {
Some(new UnionRDD(ssc.sc, rdds))
} else {
None
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
index a3ad5eaa40..a83c0d922d 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
@@ -218,12 +218,12 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging {
// Batches that were unprocessed before failure
val pendingTimes = ssc.initialCheckpoint.pendingTimes.sorted(Time.ordering)
- logInfo("Batches pending processing (" + pendingTimes.size + " batches): " +
+ logInfo("Batches pending processing (" + pendingTimes.length + " batches): " +
pendingTimes.mkString(", "))
// Reschedule jobs for these times
val timesToReschedule = (pendingTimes ++ downTimes).filter { _ < restartTime }
.distinct.sorted(Time.ordering)
- logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " +
+ logInfo("Batches to reschedule (" + timesToReschedule.length + " batches): " +
timesToReschedule.mkString(", "))
timesToReschedule.foreach { time =>
// Allocate the related blocks when recovering from failure, because some blocks that were
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
index 678f1dc950..6e24365c4e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala
@@ -131,7 +131,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
// Track the active receiver job number. When a receiver job exits ultimately, countDown will
// be called.
- private val receiverJobExitLatch = new CountDownLatch(receiverInputStreams.size)
+ private val receiverJobExitLatch = new CountDownLatch(receiverInputStreams.length)
/**
* Track all receivers' information. The key is the receiver id, the value is the receiver info.
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala
index e235afad5e..c8a2c17bbd 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala
@@ -259,7 +259,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") {
} else {
var nextLineIndex = failure.indexOf("\n")
if (nextLineIndex < 0) {
- nextLineIndex = failure.size
+ nextLineIndex = failure.length
}
val firstLine = failure.substring(0, nextLineIndex)
s"Failed due to error: $firstLine\n$failure"
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala
index 30a3a98c01..430f35a400 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala
@@ -161,7 +161,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext)
}
def numInactiveReceivers: Int = {
- ssc.graph.getReceiverInputStreams().size - numActiveReceivers
+ ssc.graph.getReceiverInputStreams().length - numActiveReceivers
}
def numTotalCompletedBatches: Long = synchronized {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
index c5d9f26cb2..2a5d0839fd 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala
@@ -444,7 +444,7 @@ private[ui] class StreamingPage(parent: StreamingTab)
}.getOrElse(emptyCell)
val receiverLastError = receiverInfo.map { info =>
val msg = s"${info.lastErrorMessage} - ${info.lastError}"
- if (msg.size > 100) msg.take(97) + "..." else msg
+ if (msg.length > 100) msg.take(97) + "..." else msg
}.getOrElse(emptyCell)
val receiverLastErrorTime = receiverInfo.map {
r => if (r.lastErrorTime < 0) "-" else SparkUIUtils.formatDate(r.lastErrorTime)
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala
index a485a46937..9b1c939e93 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala
@@ -136,7 +136,7 @@ private[streaming] object UIUtils {
} else {
var nextLineIndex = failure.indexOf("\n")
if (nextLineIndex < 0) {
- nextLineIndex = failure.size
+ nextLineIndex = failure.length
}
val firstLine = failure.substring(0, nextLineIndex)
s"Failed due to error: $firstLine\n$failure"
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index f1c64799c6..bd60059b18 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -76,7 +76,7 @@ class BasicOperationsSuite extends TestSuiteBase {
assert(numInputPartitions === 2, "Number of input partitions has been changed from 2")
val input = Seq(1 to 4, 5 to 8, 9 to 12)
val output = Seq(Seq(3, 7), Seq(11, 15), Seq(19, 23))
- val operation = (r: DStream[Int]) => r.mapPartitions(x => Iterator(x.reduce(_ + _)))
+ val operation = (r: DStream[Int]) => r.mapPartitions(x => Iterator(x.sum))
testOperation(input, operation, output, true)
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
index 6c60652cd6..19c89fcf67 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala
@@ -278,7 +278,7 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers {
/** Check if a sequence of numbers is in increasing order */
def isInIncreasingOrder(data: Iterable[Long]): Boolean = {
- !data.sliding(2).map{itr => itr.size == 2 && itr.head > itr.tail.head }.contains(true)
+ !data.sliding(2).exists { itr => itr.size == 2 && itr.head > itr.tail.head }
}
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
index 96dd4757be..3f12de38ef 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala
@@ -161,12 +161,12 @@ class UISeleniumSuite
jobLinks.size should be (4)
// Check stage progress
- findAll(cssSelector(""".stage-progress-cell""")).map(_.text).toSeq should be
- (List("1/1", "1/1", "1/1", "0/1 (1 failed)"))
+ findAll(cssSelector(""".stage-progress-cell""")).map(_.text).toList should be (
+ List("1/1", "1/1", "1/1", "0/1 (1 failed)"))
// Check job progress
- findAll(cssSelector(""".progress-cell""")).map(_.text).toSeq should be
- (List("1/1", "1/1", "1/1", "0/1 (1 failed)"))
+ findAll(cssSelector(""".progress-cell""")).map(_.text).toList should be (
+ List("4/4", "4/4", "4/4", "0/4 (1 failed)"))
// Check stacktrace
val errorCells = findAll(cssSelector(""".stacktrace-details""")).map(_.text).toSeq
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala
index 734dd93cda..7460e8629b 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala
@@ -74,12 +74,12 @@ abstract class CommonWriteAheadLogTests(
test(testPrefix + "read all logs") {
// Write data manually for testing reading through WriteAheadLog
- val writtenData = (1 to 10).map { i =>
+ val writtenData = (1 to 10).flatMap { i =>
val data = generateRandomData()
val file = testDir + s"/log-$i-$i"
writeDataManually(data, file, allowBatching)
data
- }.flatten
+ }
val logDirectoryPath = new Path(testDir)
val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf)
@@ -193,12 +193,12 @@ abstract class CommonWriteAheadLogTests(
test(testPrefix + "parallel recovery not enabled if closeFileAfterWrite = false") {
// write some data
- val writtenData = (1 to 10).map { i =>
+ val writtenData = (1 to 10).flatMap { i =>
val data = generateRandomData()
val file = testDir + s"/log-$i-$i"
writeDataManually(data, file, allowBatching)
data
- }.flatten
+ }
val wal = createWriteAheadLog(testDir, closeFileAfterWrite, allowBatching)
// create iterator but don't materialize it
diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
index 998bd1377d..19065373c6 100644
--- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
+++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala
@@ -213,7 +213,7 @@ class ClientSuite extends SparkFunSuite with Matchers with BeforeAndAfterAll
appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method =>
val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]]
tags should contain allOf ("tag1", "dup", "tag2", "multi word")
- tags.asScala.filter(_.nonEmpty).size should be (4)
+ tags.asScala.count(_.nonEmpty) should be (4)
}
appContext.getMaxAppAttempts should be (42)
}