From 20fd254101553cb5a4c932c8d03064899112bee6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun <dongjoon@apache.org> Date: Mon, 21 Mar 2016 07:58:57 +0000 Subject: [PATCH] [SPARK-14011][CORE][SQL] Enable `LineLength` Java checkstyle rule ## What changes were proposed in this pull request? [Spark Coding Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide) has 100-character limit on lines, but it's disabled for Java since 11/09/15. This PR enables **LineLength** checkstyle again. To help that, this also introduces **RedundantImport** and **RedundantModifier**, too. The following is the diff on `checkstyle.xml`. ```xml - <!-- TODO: 11/09/15 disabled - the lengths are currently > 100 in many places --> - <!-- <module name="LineLength"> <property name="max" value="100"/> <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/> </module> - --> <module name="NoLineWrap"/> <module name="EmptyBlock"> <property name="option" value="TEXT"/> -167,5 +164,7 </module> <module name="CommentsIndentation"/> <module name="UnusedImports"/> + <module name="RedundantImport"/> + <module name="RedundantModifier"/> ``` ## How was this patch tested? Currently, `lint-java` is disabled in Jenkins. It needs a manual test. After passing the Jenkins tests, `dev/lint-java` should passes locally. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #11831 from dongjoon-hyun/SPARK-14011. --- .../spark/network/TransportContext.java | 3 +- .../spark/network/client/StreamCallback.java | 6 +- .../client/TransportClientFactory.java | 2 +- .../spark/network/protocol/Message.java | 4 +- .../network/protocol/RequestMessage.java | 2 - .../network/protocol/ResponseMessage.java | 2 - .../spark/network/sasl/SaslMessage.java | 4 +- .../server/OneForOneStreamManager.java | 4 +- .../server/TransportChannelHandler.java | 4 +- .../apache/spark/network/util/ByteUnit.java | 2 +- .../util/SystemPropertyConfigProvider.java | 2 - .../network/util/TransportFrameDecoder.java | 2 +- .../network/sasl/ShuffleSecretManager.java | 1 - .../shuffle/ExternalShuffleBlockHandler.java | 3 +- .../shuffle/ExternalShuffleBlockResolver.java | 4 +- .../network/shuffle/RetryingBlockFetcher.java | 2 +- .../protocol/BlockTransferMessage.java | 4 +- .../network/sasl/SaslIntegrationSuite.java | 3 +- .../ExternalShuffleBlockHandlerSuite.java | 3 +- .../spark/util/sketch/Murmur3_x86_32.java | 2 +- .../spark/unsafe/bitset/BitSetMethods.java | 3 +- .../JavaSparkContextVarargsWorkaround.java | 3 +- .../api/java/function/DoubleFunction.java | 2 +- .../spark/api/java/function/Function2.java | 2 +- .../spark/api/java/function/Function3.java | 2 +- .../spark/api/java/function/PairFunction.java | 2 +- .../spark/memory/TaskMemoryManager.java | 6 +- .../sort/BypassMergeSortShuffleWriter.java | 2 +- .../shuffle/sort/ShuffleExternalSorter.java | 2 +- .../shuffle/sort/ShuffleInMemorySorter.java | 4 +- .../apache/spark/shuffle/sort/SpillInfo.java | 2 +- .../shuffle/sort/UnsafeShuffleWriter.java | 6 +- .../spark/status/api/v1/TaskSorting.java | 2 +- .../spark/unsafe/map/BytesToBytesMap.java | 2 +- .../unsafe/sort/UnsafeExternalSorter.java | 4 +- .../unsafe/sort/UnsafeSortDataFormat.java | 3 +- .../unsafe/sort/UnsafeSorterSpillMerger.java | 4 +- .../java/org/apache/spark/JavaAPISuite.java | 464 +++++++++--------- .../map/AbstractBytesToBytesMapSuite.java | 9 +- .../sort/UnsafeExternalSorterSuite.java | 6 +- .../sort/UnsafeInMemorySorterSuite.java | 4 +- dev/checkstyle-suppressions.xml | 10 +- dev/checkstyle.xml | 5 +- .../apache/spark/examples/JavaPageRank.java | 15 +- .../apache/spark/examples/JavaWordCount.java | 26 +- ...JavaDecisionTreeClassificationExample.java | 5 +- .../examples/ml/JavaDeveloperApiExample.java | 6 +- ...aGradientBoostedTreeClassifierExample.java | 3 +- ...avaBinaryClassificationMetricsExample.java | 3 +- .../mllib/JavaIsotonicRegressionExample.java | 6 +- .../mllib/JavaStreamingTestExample.java | 1 - .../streaming/JavaDirectKafkaWordCount.java | 3 +- .../examples/streaming/JavaQueueStream.java | 1 - .../JavaRecoverableNetworkWordCount.java | 9 +- .../JavaStatefulNetworkWordCount.java | 3 +- .../streaming/JavaKinesisWordCountASL.java | 5 +- .../launcher/AbstractCommandBuilder.java | 2 +- .../spark/launcher/CommandBuilderUtils.java | 2 +- .../apache/spark/launcher/SparkAppHandle.java | 2 +- .../JavaLogisticRegressionSuite.java | 1 - .../ml/classification/JavaOneVsRestSuite.java | 3 +- .../ml/feature/JavaStopWordsRemoverSuite.java | 3 +- .../execution/UnsafeExternalRowSorter.java | 2 +- .../UnsafeFixedWidthAggregationMap.java | 4 +- .../sql/execution/UnsafeKVExternalSorter.java | 2 +- .../execution/vectorized/ColumnVector.java | 6 +- .../execution/vectorized/ColumnarBatch.java | 44 +- .../vectorized/OffHeapColumnVector.java | 90 ++-- .../vectorized/OnHeapColumnVector.java | 92 ++-- .../spark/sql/JavaApplySchemaSuite.java | 13 +- .../apache/spark/sql/JavaDataFrameSuite.java | 3 +- .../apache/spark/sql/hive/test/Complex.java | 2 +- .../apache/spark/streaming/JavaAPISuite.java | 113 +++-- .../JavaStreamingListenerAPISuite.java | 6 +- 74 files changed, 579 insertions(+), 505 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java index 238710d172..5320b28bc0 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/common/network-common/src/main/java/org/apache/spark/network/TransportContext.java @@ -43,7 +43,8 @@ import org.apache.spark.network.util.TransportFrameDecoder; /** * Contains the context to create a {@link TransportServer}, {@link TransportClientFactory}, and to - * setup Netty Channel pipelines with a {@link org.apache.spark.network.server.TransportChannelHandler}. + * setup Netty Channel pipelines with a + * {@link org.apache.spark.network.server.TransportChannelHandler}. * * There are two communication protocols that the TransportClient provides, control-plane RPCs and * data-plane "chunk fetching". The handling of the RPCs is performed outside of the scope of the diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/StreamCallback.java b/common/network-common/src/main/java/org/apache/spark/network/client/StreamCallback.java index 29e6a30dc1..d322aec287 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/StreamCallback.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/StreamCallback.java @@ -21,9 +21,9 @@ import java.io.IOException; import java.nio.ByteBuffer; /** - * Callback for streaming data. Stream data will be offered to the {@link #onData(String, ByteBuffer)} - * method as it arrives. Once all the stream data is received, {@link #onComplete(String)} will be - * called. + * Callback for streaming data. Stream data will be offered to the + * {@link #onData(String, ByteBuffer)} method as it arrives. Once all the stream data is received, + * {@link #onComplete(String)} will be called. * <p> * The network library guarantees that a single thread will call these methods at a time, but * different call may be made by different threads. diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 1008c67de3..f179bad1f4 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -64,7 +64,7 @@ public class TransportClientFactory implements Closeable { TransportClient[] clients; Object[] locks; - public ClientPool(int size) { + ClientPool(int size) { clients = new TransportClient[size]; locks = new Object[size]; for (int i = 0; i < size; i++) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java index 66f5b8b3a5..434935a8ef 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java @@ -33,7 +33,7 @@ public interface Message extends Encodable { boolean isBodyInFrame(); /** Preceding every serialized Message is its type, which allows us to deserialize it. */ - public static enum Type implements Encodable { + enum Type implements Encodable { ChunkFetchRequest(0), ChunkFetchSuccess(1), ChunkFetchFailure(2), RpcRequest(3), RpcResponse(4), RpcFailure(5), StreamRequest(6), StreamResponse(7), StreamFailure(8), @@ -41,7 +41,7 @@ public interface Message extends Encodable { private final byte id; - private Type(int id) { + Type(int id) { assert id < 128 : "Cannot have more than 128 message types"; this.id = (byte) id; } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java index 31b15bb17a..b85171ed6f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java @@ -17,8 +17,6 @@ package org.apache.spark.network.protocol; -import org.apache.spark.network.protocol.Message; - /** Messages from the client to the server. */ public interface RequestMessage extends Message { // token interface diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java index 6edffd11cf..194e6d9aa2 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java @@ -17,8 +17,6 @@ package org.apache.spark.network.protocol; -import org.apache.spark.network.protocol.Message; - /** Messages from the server to the client. */ public interface ResponseMessage extends Message { // token interface diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java index e52b526f09..7331c2b481 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java @@ -36,11 +36,11 @@ class SaslMessage extends AbstractMessage { public final String appId; - public SaslMessage(String appId, byte[] message) { + SaslMessage(String appId, byte[] message) { this(appId, Unpooled.wrappedBuffer(message)); } - public SaslMessage(String appId, ByteBuf message) { + SaslMessage(String appId, ByteBuf message) { super(new NettyManagedBuffer(message), true); this.appId = appId; } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index ea9e735e0a..e2222ae085 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -32,8 +32,8 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.TransportClient; /** - * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually - * fetched as chunks by the client. Each registered buffer is one chunk. + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are + * individually fetched as chunks by the client. Each registered buffer is one chunk. */ public class OneForOneStreamManager extends StreamManager { private final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java index 18a9b7887e..f2223379a9 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -141,8 +141,8 @@ public class TransportChannelHandler extends SimpleChannelInboundHandler<Message if (responseHandler.numOutstandingRequests() > 0) { String address = NettyUtils.getRemoteAddress(ctx.channel()); logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + - "requests. Assuming connection is dead; please adjust spark.network.timeout if this " + - "is wrong.", address, requestTimeoutNs / 1000 / 1000); + "requests. Assuming connection is dead; please adjust spark.network.timeout if " + + "this is wrong.", address, requestTimeoutNs / 1000 / 1000); client.timeOut(); ctx.close(); } else if (closeIdleConnections) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java index a2f018373f..e097714bbc 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java @@ -24,7 +24,7 @@ public enum ByteUnit { TiB ((long) Math.pow(1024L, 4L)), PiB ((long) Math.pow(1024L, 5L)); - private ByteUnit(long multiplier) { + ByteUnit(long multiplier) { this.multiplier = multiplier; } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java index 5f20b70678..f15ec8d294 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java @@ -19,8 +19,6 @@ package org.apache.spark.network.util; import java.util.NoSuchElementException; -import org.apache.spark.network.util.ConfigProvider; - /** Uses System properties to obtain config values. */ public class SystemPropertyConfigProvider extends ConfigProvider { @Override diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java index 3f7024a6aa..bd1830e6ab 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java @@ -205,7 +205,7 @@ public class TransportFrameDecoder extends ChannelInboundHandlerAdapter { return interceptor != null; } - public static interface Interceptor { + public interface Interceptor { /** * Handles data received from the remote end. diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java index cdce297233..268cb40121 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java @@ -17,7 +17,6 @@ package org.apache.spark.network.sasl; -import java.lang.Override; import java.nio.ByteBuffer; import java.util.concurrent.ConcurrentHashMap; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index f22187a01d..f8d03b3b94 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -52,7 +52,8 @@ public class ExternalShuffleBlockHandler extends RpcHandler { final ExternalShuffleBlockResolver blockManager; private final OneForOneStreamManager streamManager; - public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { + public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFile) + throws IOException { this(new OneForOneStreamManager(), new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 460110d78f..ce5c68e853 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -423,7 +423,9 @@ public class ExternalShuffleBlockResolver { public final int major; public final int minor; - @JsonCreator public StoreVersion(@JsonProperty("major") int major, @JsonProperty("minor") int minor) { + @JsonCreator public StoreVersion( + @JsonProperty("major") int major, + @JsonProperty("minor") int minor) { this.major = major; this.minor = minor; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index 4bb0498e5d..d81cf869dd 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -46,7 +46,7 @@ public class RetryingBlockFetcher { * Used to initiate the first fetch for all blocks, and subsequently for retrying the fetch on any * remaining blocks. */ - public static interface BlockFetchStarter { + public interface BlockFetchStarter { /** * Creates a new BlockFetcher to fetch the given block ids which may do some synchronous * bootstrapping followed by fully asynchronous block fetching. diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index 21c0ff4136..9af6759f5d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -40,13 +40,13 @@ public abstract class BlockTransferMessage implements Encodable { protected abstract Type type(); /** Preceding every serialized message is its type, which allows us to deserialize it. */ - public static enum Type { + public enum Type { OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), HEARTBEAT(5); private final byte id; - private Type(int id) { + Type(int id) { assert id < 128 : "Cannot have more than 128 message types"; this.id = (byte) id; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index 5322fcd781..5bf9924185 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -212,7 +212,8 @@ public class SaslIntegrationSuite { }; String[] blockIds = { "shuffle_2_3_4", "shuffle_6_7_8" }; - OneForOneBlockFetcher fetcher = new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener); + OneForOneBlockFetcher fetcher = + new OneForOneBlockFetcher(client1, "app-2", "0", blockIds, listener); fetcher.start(); blockFetchLatch.await(); checkSecurityException(exception.get()); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 9379412155..c2e0b7447f 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -113,7 +113,8 @@ public class ExternalShuffleBlockHandlerSuite { // pass } - ByteBuffer unexpectedMsg = new UploadBlock("a", "e", "b", new byte[1], new byte[2]).toByteBuffer(); + ByteBuffer unexpectedMsg = new UploadBlock("a", "e", "b", new byte[1], + new byte[2]).toByteBuffer(); try { handler.receive(client, unexpectedMsg, callback); fail("Should have thrown"); diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java index 3d1f28bcb9..a61ce4fb72 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/Murmur3_x86_32.java @@ -28,7 +28,7 @@ final class Murmur3_x86_32 { private final int seed; - public Murmur3_x86_32(int seed) { + Murmur3_x86_32(int seed) { this.seed = seed; } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java index 7857bf66a7..c8c57381f3 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/bitset/BitSetMethods.java @@ -87,7 +87,8 @@ public final class BitSetMethods { * To iterate over the true bits in a BitSet, use the following loop: * <pre> * <code> - * for (long i = bs.nextSetBit(0, sizeInWords); i >= 0; i = bs.nextSetBit(i + 1, sizeInWords)) { + * for (long i = bs.nextSetBit(0, sizeInWords); i >= 0; + * i = bs.nextSetBit(i + 1, sizeInWords)) { * // operate on index i here * } * </code> diff --git a/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java index d4c42b38ac..0dd8fafbf2 100644 --- a/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java @@ -62,5 +62,6 @@ abstract class JavaSparkContextVarargsWorkaround { // These methods take separate "first" and "rest" elements to avoid having the same type erasure public abstract <T> JavaRDD<T> union(JavaRDD<T> first, List<JavaRDD<T>> rest); public abstract JavaDoubleRDD union(JavaDoubleRDD first, List<JavaDoubleRDD> rest); - public abstract <K, V> JavaPairRDD<K, V> union(JavaPairRDD<K, V> first, List<JavaPairRDD<K, V>> rest); + public abstract <K, V> JavaPairRDD<K, V> union(JavaPairRDD<K, V> first, List<JavaPairRDD<K, V>> + rest); } diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java index 150144e0e4..bf16f791f9 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java @@ -23,5 +23,5 @@ import java.io.Serializable; * A function that returns Doubles, and can be used to construct DoubleRDDs. */ public interface DoubleFunction<T> extends Serializable { - public double call(T t) throws Exception; + double call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function2.java b/core/src/main/java/org/apache/spark/api/java/function/Function2.java index 793caaa61a..a975ce3c68 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/Function2.java +++ b/core/src/main/java/org/apache/spark/api/java/function/Function2.java @@ -23,5 +23,5 @@ import java.io.Serializable; * A two-argument function that takes arguments of type T1 and T2 and returns an R. */ public interface Function2<T1, T2, R> extends Serializable { - public R call(T1 v1, T2 v2) throws Exception; + R call(T1 v1, T2 v2) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/Function3.java b/core/src/main/java/org/apache/spark/api/java/function/Function3.java index b4151c3417..6eecfb645a 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/Function3.java +++ b/core/src/main/java/org/apache/spark/api/java/function/Function3.java @@ -23,5 +23,5 @@ import java.io.Serializable; * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R. */ public interface Function3<T1, T2, T3, R> extends Serializable { - public R call(T1 v1, T2 v2, T3 v3) throws Exception; + R call(T1 v1, T2 v2, T3 v3) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java index 99bf240a17..2fdfa7184a 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java @@ -26,5 +26,5 @@ import scala.Tuple2; * construct PairRDDs. */ public interface PairFunction<T, K, V> extends Serializable { - public Tuple2<K, V> call(T t) throws Exception; + Tuple2<K, V> call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 8757dff36f..18612dd68c 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -67,9 +67,9 @@ public class TaskMemoryManager { /** * Maximum supported data page size (in bytes). In principle, the maximum addressable page size is - * (1L << OFFSET_BITS) bytes, which is 2+ petabytes. However, the on-heap allocator's maximum page - * size is limited by the maximum amount of data that can be stored in a long[] array, which is - * (2^32 - 1) * 8 bytes (or 16 gigabytes). Therefore, we cap this at 16 gigabytes. + * (1L << OFFSET_BITS) bytes, which is 2+ petabytes. However, the on-heap allocator's + * maximum page size is limited by the maximum amount of data that can be stored in a long[] + * array, which is (2^32 - 1) * 8 bytes (or 16 gigabytes). Therefore, we cap this at 16 gigabytes. */ public static final long MAXIMUM_PAGE_SIZE_BYTES = ((1L << 31) - 1) * 8L; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 052be54d8c..7a60c3eb35 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -98,7 +98,7 @@ final class BypassMergeSortShuffleWriter<K, V> extends ShuffleWriter<K, V> { */ private boolean stopping = false; - public BypassMergeSortShuffleWriter( + BypassMergeSortShuffleWriter( BlockManager blockManager, IndexShuffleBlockResolver shuffleBlockResolver, BypassMergeSortShuffleHandle<K, V> handle, diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index c7d89e6b09..81ee7ab58a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -96,7 +96,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { @Nullable private MemoryBlock currentPage = null; private long pageCursor = -1; - public ShuffleExternalSorter( + ShuffleExternalSorter( TaskMemoryManager memoryManager, BlockManager blockManager, TaskContext taskContext, 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 2381cff61f..fe79ff0e30 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 @@ -51,7 +51,7 @@ final class ShuffleInMemorySorter { */ private int pos = 0; - public ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize) { + ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize) { this.consumer = consumer; assert (initialSize > 0); this.array = consumer.allocateArray(initialSize); @@ -122,7 +122,7 @@ final class ShuffleInMemorySorter { final PackedRecordPointer packedRecordPointer = new PackedRecordPointer(); private int position = 0; - public ShuffleSorterIterator(int numRecords, LongArray pointerArray) { + ShuffleSorterIterator(int numRecords, LongArray pointerArray) { this.numRecords = numRecords; this.pointerArray = pointerArray; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java b/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java index df9f7b7abe..865def6b83 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/SpillInfo.java @@ -29,7 +29,7 @@ final class SpillInfo { final File file; final TempShuffleBlockId blockId; - public SpillInfo(int numPartitions, File file, TempShuffleBlockId blockId) { + SpillInfo(int numPartitions, File file, TempShuffleBlockId blockId) { this.partitionLengths = new long[numPartitions]; this.file = file; this.blockId = blockId; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index cd06ce9fb9..0c5fb883a8 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -45,7 +45,6 @@ import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.SerializationStream; -import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; @@ -82,7 +81,7 @@ public class UnsafeShuffleWriter<K, V> extends ShuffleWriter<K, V> { /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream { - public MyByteArrayOutputStream(int size) { super(size); } + MyByteArrayOutputStream(int size) { super(size); } public byte[] getBuf() { return buf; } } @@ -108,7 +107,8 @@ public class UnsafeShuffleWriter<K, V> extends ShuffleWriter<K, V> { if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) { throw new IllegalArgumentException( "UnsafeShuffleWriter can only be used for shuffles with at most " + - SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE() + " reduce partitions"); + SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE() + + " reduce partitions"); } this.blockManager = blockManager; this.shuffleBlockResolver = shuffleBlockResolver; diff --git a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java index 0cf84d5f9b..9307eb93a5 100644 --- a/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java +++ b/core/src/main/java/org/apache/spark/status/api/v1/TaskSorting.java @@ -28,7 +28,7 @@ public enum TaskSorting { DECREASING_RUNTIME("-runtime"); private final Set<String> alternateNames; - private TaskSorting(String... names) { + TaskSorting(String... names) { alternateNames = new HashSet<>(); for (String n: names) { alternateNames.add(n); diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index b55a322a1b..de36814ecc 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -689,7 +689,7 @@ public final class BytesToBytesMap extends MemoryConsumer { offset += keyLength; Platform.copyMemory(valueBase, valueOffset, base, offset, valueLength); - // --- Update bookkeeping data structures ----------------------------------------------------- + // --- Update bookkeeping data structures ---------------------------------------------------- offset = currentPage.getBaseOffset(); Platform.putInt(base, offset, Platform.getInt(base, offset) + 1); pageCursor += recordLength; 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 db8e7a759a..927b19c4e8 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 @@ -432,7 +432,7 @@ public final class UnsafeExternalSorter extends MemoryConsumer { private boolean loaded = false; private int numRecords = 0; - public SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) { + SpillableIterator(UnsafeInMemorySorter.SortedIterator inMemIterator) { this.upstream = inMemIterator; this.numRecords = inMemIterator.getNumRecords(); } @@ -567,7 +567,7 @@ public final class UnsafeExternalSorter extends MemoryConsumer { private UnsafeSorterIterator current; private int numRecords; - public ChainedIterator(Queue<UnsafeSorterIterator> iterators) { + ChainedIterator(Queue<UnsafeSorterIterator> iterators) { assert iterators.size() > 0; this.numRecords = 0; for (UnsafeSorterIterator iter: iterators) { diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java index d3137f5f31..12fb62fb77 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java @@ -47,7 +47,8 @@ final class UnsafeSortDataFormat extends SortDataFormat<RecordPointerAndKeyPrefi } @Override - public RecordPointerAndKeyPrefix getKey(LongArray data, int pos, RecordPointerAndKeyPrefix reuse) { + public RecordPointerAndKeyPrefix getKey(LongArray data, int pos, + RecordPointerAndKeyPrefix reuse) { reuse.recordPointer = data.get(pos * 2); reuse.keyPrefix = data.get(pos * 2 + 1); return reuse; diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java index ceb59352af..2b1c860e55 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java @@ -26,7 +26,7 @@ final class UnsafeSorterSpillMerger { private int numRecords = 0; private final PriorityQueue<UnsafeSorterIterator> priorityQueue; - public UnsafeSorterSpillMerger( + UnsafeSorterSpillMerger( final RecordComparator recordComparator, final PrefixComparator prefixComparator, final int numSpills) { @@ -57,7 +57,7 @@ final class UnsafeSorterSpillMerger { // make sure the hasNext method of UnsafeSorterIterator returned by getSortedIterator // does not return wrong result because hasNext will returns true // at least priorityQueue.size() times. If we allow n spillReaders in the - // priorityQueue, we will have n extra empty records in the result of the UnsafeSorterIterator. + // priorityQueue, we will have n extra empty records in the result of UnsafeSorterIterator. spillReader.loadNext(); priorityQueue.add(spillReader); numRecords += spillReader.getNumRecords(); diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index c1036b8fac..0f65554516 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -54,7 +54,7 @@ import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.hadoop.mapreduce.Job; import org.junit.After; -import org.junit.Assert; +import static org.junit.Assert.*; import org.junit.Before; import org.junit.Test; @@ -102,19 +102,19 @@ public class JavaAPISuite implements Serializable { JavaRDD<String> s2 = sc.parallelize(strings); // Varargs JavaRDD<String> sUnion = sc.union(s1, s2); - Assert.assertEquals(4, sUnion.count()); + assertEquals(4, sUnion.count()); // List List<JavaRDD<String>> list = new ArrayList<>(); list.add(s2); sUnion = sc.union(s1, list); - Assert.assertEquals(4, sUnion.count()); + assertEquals(4, sUnion.count()); // Union of JavaDoubleRDDs List<Double> doubles = Arrays.asList(1.0, 2.0); JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles); JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles); JavaDoubleRDD dUnion = sc.union(d1, d2); - Assert.assertEquals(4, dUnion.count()); + assertEquals(4, dUnion.count()); // Union of JavaPairRDDs List<Tuple2<Integer, Integer>> pairs = new ArrayList<>(); @@ -123,7 +123,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<Integer, Integer> p1 = sc.parallelizePairs(pairs); JavaPairRDD<Integer, Integer> p2 = sc.parallelizePairs(pairs); JavaPairRDD<Integer, Integer> pUnion = sc.union(p1, p2); - Assert.assertEquals(4, pUnion.count()); + assertEquals(4, pUnion.count()); } @SuppressWarnings("unchecked") @@ -135,17 +135,17 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> s2 = sc.parallelize(ints2); JavaRDD<Integer> intersections = s1.intersection(s2); - Assert.assertEquals(3, intersections.count()); + assertEquals(3, intersections.count()); JavaRDD<Integer> empty = sc.emptyRDD(); JavaRDD<Integer> emptyIntersection = empty.intersection(s2); - Assert.assertEquals(0, emptyIntersection.count()); + assertEquals(0, emptyIntersection.count()); List<Double> doubles = Arrays.asList(1.0, 2.0); JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles); JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles); JavaDoubleRDD dIntersection = d1.intersection(d2); - Assert.assertEquals(2, dIntersection.count()); + assertEquals(2, dIntersection.count()); List<Tuple2<Integer, Integer>> pairs = new ArrayList<>(); pairs.add(new Tuple2<>(1, 2)); @@ -153,7 +153,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<Integer, Integer> p1 = sc.parallelizePairs(pairs); JavaPairRDD<Integer, Integer> p2 = sc.parallelizePairs(pairs); JavaPairRDD<Integer, Integer> pIntersection = p1.intersection(p2); - Assert.assertEquals(2, pIntersection.count()); + assertEquals(2, pIntersection.count()); } @Test @@ -162,9 +162,9 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> rdd = sc.parallelize(ints); // the seeds here are "magic" to make this work out nicely JavaRDD<Integer> sample20 = rdd.sample(true, 0.2, 8); - Assert.assertEquals(2, sample20.count()); + assertEquals(2, sample20.count()); JavaRDD<Integer> sample20WithoutReplacement = rdd.sample(false, 0.2, 2); - Assert.assertEquals(2, sample20WithoutReplacement.count()); + assertEquals(2, sample20WithoutReplacement.count()); } @Test @@ -176,13 +176,13 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> rdd = sc.parallelize(ints); JavaRDD<Integer>[] splits = rdd.randomSplit(new double[] { 0.4, 0.6, 1.0 }, 31); // the splits aren't perfect -- not enough data for them to be -- just check they're about right - Assert.assertEquals(3, splits.length); + assertEquals(3, splits.length); long s0 = splits[0].count(); long s1 = splits[1].count(); long s2 = splits[2].count(); - Assert.assertTrue(s0 + " not within expected range", s0 > 150 && s0 < 250); - Assert.assertTrue(s1 + " not within expected range", s1 > 250 && s0 < 350); - Assert.assertTrue(s2 + " not within expected range", s2 > 430 && s2 < 570); + assertTrue(s0 + " not within expected range", s0 > 150 && s0 < 250); + assertTrue(s1 + " not within expected range", s1 > 250 && s0 < 350); + assertTrue(s2 + " not within expected range", s2 > 430 && s2 < 570); } @Test @@ -196,17 +196,17 @@ public class JavaAPISuite implements Serializable { // Default comparator JavaPairRDD<Integer, Integer> sortedRDD = rdd.sortByKey(); - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); // Custom comparator sortedRDD = rdd.sortByKey(Collections.<Integer>reverseOrder(), false); - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); } @SuppressWarnings("unchecked") @@ -235,19 +235,19 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<Integer, Integer> repartitioned = rdd.repartitionAndSortWithinPartitions(partitioner); - Assert.assertTrue(repartitioned.partitioner().isPresent()); - Assert.assertEquals(repartitioned.partitioner().get(), partitioner); + assertTrue(repartitioned.partitioner().isPresent()); + assertEquals(repartitioned.partitioner().get(), partitioner); List<List<Tuple2<Integer, Integer>>> partitions = repartitioned.glom().collect(); - Assert.assertEquals(partitions.get(0), + assertEquals(partitions.get(0), Arrays.asList(new Tuple2<>(0, 5), new Tuple2<>(0, 8), new Tuple2<>(2, 6))); - Assert.assertEquals(partitions.get(1), + assertEquals(partitions.get(1), Arrays.asList(new Tuple2<>(1, 3), new Tuple2<>(3, 8), new Tuple2<>(3, 8))); } @Test public void emptyRDD() { JavaRDD<String> rdd = sc.emptyRDD(); - Assert.assertEquals("Empty RDD shouldn't have any values", 0, rdd.count()); + assertEquals("Empty RDD shouldn't have any values", 0, rdd.count()); } @Test @@ -260,17 +260,18 @@ public class JavaAPISuite implements Serializable { JavaRDD<Tuple2<Integer, Integer>> rdd = sc.parallelize(pairs); // compare on first value - JavaRDD<Tuple2<Integer, Integer>> sortedRDD = rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() { + JavaRDD<Tuple2<Integer, Integer>> sortedRDD = + rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() { @Override public Integer call(Tuple2<Integer, Integer> t) { return t._1(); } }, true, 2); - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); // compare on second value sortedRDD = rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() { @@ -279,10 +280,10 @@ public class JavaAPISuite implements Serializable { return t._2(); } }, true, 2); - Assert.assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); + assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); - Assert.assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1)); - Assert.assertEquals(new Tuple2<>(0, 4), sortedPairs.get(2)); + assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1)); + assertEquals(new Tuple2<>(0, 4), sortedPairs.get(2)); } @Test @@ -295,7 +296,7 @@ public class JavaAPISuite implements Serializable { accum.add(1); } }); - Assert.assertEquals(2, accum.value().intValue()); + assertEquals(2, accum.value().intValue()); } @Test @@ -311,7 +312,7 @@ public class JavaAPISuite implements Serializable { } } }); - Assert.assertEquals(2, accum.value().intValue()); + assertEquals(2, accum.value().intValue()); } @Test @@ -319,7 +320,7 @@ public class JavaAPISuite implements Serializable { List<Integer> correct = Arrays.asList(1, 2, 3, 4); JavaRDD<Integer> rdd = sc.parallelize(correct); List<Integer> result = Lists.newArrayList(rdd.toLocalIterator()); - Assert.assertEquals(correct, result); + assertEquals(correct, result); } @Test @@ -327,7 +328,7 @@ public class JavaAPISuite implements Serializable { List<Integer> dataArray = Arrays.asList(1, 2, 3, 4); JavaPairRDD<Integer, Long> zip = sc.parallelize(dataArray).zipWithUniqueId(); JavaRDD<Long> indexes = zip.values(); - Assert.assertEquals(4, new HashSet<>(indexes.collect()).size()); + assertEquals(4, new HashSet<>(indexes.collect()).size()); } @Test @@ -336,7 +337,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<Integer, Long> zip = sc.parallelize(dataArray).zipWithIndex(); JavaRDD<Long> indexes = zip.values(); List<Long> correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); - Assert.assertEquals(correctIndexes, indexes.collect()); + assertEquals(correctIndexes, indexes.collect()); } @SuppressWarnings("unchecked") @@ -347,8 +348,8 @@ public class JavaAPISuite implements Serializable { new Tuple2<>("Oranges", "Fruit"), new Tuple2<>("Oranges", "Citrus") )); - Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); + assertEquals(2, categories.lookup("Oranges").size()); + assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -361,14 +362,14 @@ public class JavaAPISuite implements Serializable { } }; JavaPairRDD<Boolean, Iterable<Integer>> oddsAndEvens = rdd.groupBy(isOdd); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @Test @@ -384,14 +385,14 @@ public class JavaAPISuite implements Serializable { }; JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd); JavaPairRDD<Boolean, Iterable<Tuple2<Integer, Integer>>> oddsAndEvens = pairRDD.groupBy(areOdd); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = pairRDD.groupBy(areOdd, 1); - Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens - Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + assertEquals(2, oddsAndEvens.count()); + assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -408,8 +409,8 @@ public class JavaAPISuite implements Serializable { }; JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd); JavaPairRDD<String, Tuple2<Integer, Integer>> keyed = pairRDD.keyBy(sumToString); - Assert.assertEquals(7, keyed.count()); - Assert.assertEquals(1, (long) keyed.lookup("2").get(0)._1()); + assertEquals(7, keyed.count()); + assertEquals(1, (long) keyed.lookup("2").get(0)._1()); } @SuppressWarnings("unchecked") @@ -426,8 +427,8 @@ public class JavaAPISuite implements Serializable { )); JavaPairRDD<String, Tuple2<Iterable<String>, Iterable<Integer>>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); } @@ -451,9 +452,9 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<String, Tuple3<Iterable<String>, Iterable<Integer>, Iterable<Integer>>> cogrouped = categories.cogroup(prices, quantities); - Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); - Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); + assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); cogrouped.collect(); @@ -480,12 +481,12 @@ public class JavaAPISuite implements Serializable { new Tuple2<>("Apples", "US") )); - JavaPairRDD<String, Tuple4<Iterable<String>, Iterable<Integer>, Iterable<Integer>, Iterable<String>>> cogrouped = - categories.cogroup(prices, quantities, countries); - Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); - Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); - Assert.assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); - Assert.assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4())); + JavaPairRDD<String, Tuple4<Iterable<String>, Iterable<Integer>, Iterable<Integer>, + Iterable<String>>> cogrouped = categories.cogroup(prices, quantities, countries); + assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); + assertEquals("[42]", Iterables.toString(cogrouped.lookup("Apples").get(0)._3())); + assertEquals("[BR]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._4())); cogrouped.collect(); } @@ -507,7 +508,7 @@ public class JavaAPISuite implements Serializable { )); List<Tuple2<Integer,Tuple2<Integer,Optional<Character>>>> joined = rdd1.leftOuterJoin(rdd2).collect(); - Assert.assertEquals(5, joined.size()); + assertEquals(5, joined.size()); Tuple2<Integer,Tuple2<Integer,Optional<Character>>> firstUnmatched = rdd1.leftOuterJoin(rdd2).filter( new Function<Tuple2<Integer, Tuple2<Integer, Optional<Character>>>, Boolean>() { @@ -516,7 +517,7 @@ public class JavaAPISuite implements Serializable { return !tup._2()._2().isPresent(); } }).first(); - Assert.assertEquals(3, firstUnmatched._1().intValue()); + assertEquals(3, firstUnmatched._1().intValue()); } @Test @@ -530,10 +531,10 @@ public class JavaAPISuite implements Serializable { }; int sum = rdd.fold(0, add); - Assert.assertEquals(33, sum); + assertEquals(33, sum); sum = rdd.reduce(add); - Assert.assertEquals(33, sum); + assertEquals(33, sum); } @Test @@ -547,7 +548,7 @@ public class JavaAPISuite implements Serializable { }; for (int depth = 1; depth <= 10; depth++) { int sum = rdd.treeReduce(add, depth); - Assert.assertEquals(-5, sum); + assertEquals(-5, sum); } } @@ -562,7 +563,7 @@ public class JavaAPISuite implements Serializable { }; for (int depth = 1; depth <= 10; depth++) { int sum = rdd.treeAggregate(0, add, add, depth); - Assert.assertEquals(-5, sum); + assertEquals(-5, sum); } } @@ -592,10 +593,10 @@ public class JavaAPISuite implements Serializable { return a; } }).collectAsMap(); - Assert.assertEquals(3, sets.size()); - Assert.assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1)); - Assert.assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3)); - Assert.assertEquals(new HashSet<>(Arrays.asList(1, 3)), sets.get(5)); + assertEquals(3, sets.size()); + assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1)); + assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3)); + assertEquals(new HashSet<>(Arrays.asList(1, 3)), sets.get(5)); } @SuppressWarnings("unchecked") @@ -616,9 +617,9 @@ public class JavaAPISuite implements Serializable { return a + b; } }); - Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); - Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); - Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); + assertEquals(1, sums.lookup(1).get(0).intValue()); + assertEquals(2, sums.lookup(2).get(0).intValue()); + assertEquals(3, sums.lookup(3).get(0).intValue()); } @SuppressWarnings("unchecked") @@ -639,14 +640,14 @@ public class JavaAPISuite implements Serializable { return a + b; } }); - Assert.assertEquals(1, counts.lookup(1).get(0).intValue()); - Assert.assertEquals(2, counts.lookup(2).get(0).intValue()); - Assert.assertEquals(3, counts.lookup(3).get(0).intValue()); + assertEquals(1, counts.lookup(1).get(0).intValue()); + assertEquals(2, counts.lookup(2).get(0).intValue()); + assertEquals(3, counts.lookup(3).get(0).intValue()); Map<Integer, Integer> localCounts = counts.collectAsMap(); - Assert.assertEquals(1, localCounts.get(1).intValue()); - Assert.assertEquals(2, localCounts.get(2).intValue()); - Assert.assertEquals(3, localCounts.get(3).intValue()); + assertEquals(1, localCounts.get(1).intValue()); + assertEquals(2, localCounts.get(2).intValue()); + assertEquals(3, localCounts.get(3).intValue()); localCounts = rdd.reduceByKeyLocally(new Function2<Integer, Integer, Integer>() { @Override @@ -654,45 +655,45 @@ public class JavaAPISuite implements Serializable { return a + b; } }); - Assert.assertEquals(1, localCounts.get(1).intValue()); - Assert.assertEquals(2, localCounts.get(2).intValue()); - Assert.assertEquals(3, localCounts.get(3).intValue()); + assertEquals(1, localCounts.get(1).intValue()); + assertEquals(2, localCounts.get(2).intValue()); + assertEquals(3, localCounts.get(3).intValue()); } @Test public void approximateResults() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Map<Integer, Long> countsByValue = rdd.countByValue(); - Assert.assertEquals(2, countsByValue.get(1).longValue()); - Assert.assertEquals(1, countsByValue.get(13).longValue()); + assertEquals(2, countsByValue.get(1).longValue()); + assertEquals(1, countsByValue.get(13).longValue()); PartialResult<Map<Integer, BoundedDouble>> approx = rdd.countByValueApprox(1); Map<Integer, BoundedDouble> finalValue = approx.getFinalValue(); - Assert.assertEquals(2.0, finalValue.get(1).mean(), 0.01); - Assert.assertEquals(1.0, finalValue.get(13).mean(), 0.01); + assertEquals(2.0, finalValue.get(1).mean(), 0.01); + assertEquals(1.0, finalValue.get(13).mean(), 0.01); } @Test public void take() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Assert.assertEquals(1, rdd.first().intValue()); + assertEquals(1, rdd.first().intValue()); rdd.take(2); rdd.takeSample(false, 2, 42); } @Test public void isEmpty() { - Assert.assertTrue(sc.emptyRDD().isEmpty()); - Assert.assertTrue(sc.parallelize(new ArrayList<Integer>()).isEmpty()); - Assert.assertFalse(sc.parallelize(Arrays.asList(1)).isEmpty()); - Assert.assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter( + assertTrue(sc.emptyRDD().isEmpty()); + assertTrue(sc.parallelize(new ArrayList<Integer>()).isEmpty()); + assertFalse(sc.parallelize(Arrays.asList(1)).isEmpty()); + assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter( new Function<Integer,Boolean>() { @Override public Boolean call(Integer i) { return i < 0; } }).isEmpty()); - Assert.assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter( + assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter( new Function<Integer, Boolean>() { @Override public Boolean call(Integer i) { @@ -706,35 +707,35 @@ public class JavaAPISuite implements Serializable { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); JavaRDD<String> stringRDD = sc.parallelize(Arrays.asList("Hello", "World")); JavaPairRDD<String, Double> cartesian = stringRDD.cartesian(doubleRDD); - Assert.assertEquals(new Tuple2<>("Hello", 1.0), cartesian.first()); + assertEquals(new Tuple2<>("Hello", 1.0), cartesian.first()); } @Test public void javaDoubleRDD() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); JavaDoubleRDD distinct = rdd.distinct(); - Assert.assertEquals(5, distinct.count()); + assertEquals(5, distinct.count()); JavaDoubleRDD filter = rdd.filter(new Function<Double, Boolean>() { @Override public Boolean call(Double x) { return x > 2.0; } }); - Assert.assertEquals(3, filter.count()); + assertEquals(3, filter.count()); JavaDoubleRDD union = rdd.union(rdd); - Assert.assertEquals(12, union.count()); + assertEquals(12, union.count()); union = union.cache(); - Assert.assertEquals(12, union.count()); + assertEquals(12, union.count()); - Assert.assertEquals(20, rdd.sum(), 0.01); + assertEquals(20, rdd.sum(), 0.01); StatCounter stats = rdd.stats(); - Assert.assertEquals(20, stats.sum(), 0.01); - Assert.assertEquals(20/6.0, rdd.mean(), 0.01); - Assert.assertEquals(20/6.0, rdd.mean(), 0.01); - Assert.assertEquals(6.22222, rdd.variance(), 0.01); - Assert.assertEquals(7.46667, rdd.sampleVariance(), 0.01); - Assert.assertEquals(2.49444, rdd.stdev(), 0.01); - Assert.assertEquals(2.73252, rdd.sampleStdev(), 0.01); + assertEquals(20, stats.sum(), 0.01); + assertEquals(20/6.0, rdd.mean(), 0.01); + assertEquals(20/6.0, rdd.mean(), 0.01); + assertEquals(6.22222, rdd.variance(), 0.01); + assertEquals(7.46667, rdd.sampleVariance(), 0.01); + assertEquals(2.49444, rdd.stdev(), 0.01); + assertEquals(2.73252, rdd.sampleStdev(), 0.01); rdd.first(); rdd.take(5); @@ -747,13 +748,13 @@ public class JavaAPISuite implements Serializable { Tuple2<double[], long[]> results = rdd.histogram(2); double[] expected_buckets = {1.0, 2.5, 4.0}; long[] expected_counts = {2, 2}; - Assert.assertArrayEquals(expected_buckets, results._1(), 0.1); - Assert.assertArrayEquals(expected_counts, results._2()); + assertArrayEquals(expected_buckets, results._1(), 0.1); + assertArrayEquals(expected_counts, results._2()); // Test with provided buckets long[] histogram = rdd.histogram(expected_buckets); - Assert.assertArrayEquals(expected_counts, histogram); + assertArrayEquals(expected_counts, histogram); // SPARK-5744 - Assert.assertArrayEquals( + assertArrayEquals( new long[] {0}, sc.parallelizeDoubles(new ArrayList<Double>(0), 1).histogram(new double[]{0.0, 1.0})); } @@ -769,42 +770,42 @@ public class JavaAPISuite implements Serializable { public void max() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); double max = rdd.max(new DoubleComparator()); - Assert.assertEquals(4.0, max, 0.001); + assertEquals(4.0, max, 0.001); } @Test public void min() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); double max = rdd.min(new DoubleComparator()); - Assert.assertEquals(1.0, max, 0.001); + assertEquals(1.0, max, 0.001); } @Test public void naturalMax() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); double max = rdd.max(); - Assert.assertEquals(4.0, max, 0.0); + assertEquals(4.0, max, 0.0); } @Test public void naturalMin() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); double max = rdd.min(); - Assert.assertEquals(1.0, max, 0.0); + assertEquals(1.0, max, 0.0); } @Test public void takeOrdered() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator())); - Assert.assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2)); + assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2, new DoubleComparator())); + assertEquals(Arrays.asList(1.0, 2.0), rdd.takeOrdered(2)); } @Test public void top() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); List<Integer> top2 = rdd.top(2); - Assert.assertEquals(Arrays.asList(4, 3), top2); + assertEquals(Arrays.asList(4, 3), top2); } private static class AddInts implements Function2<Integer, Integer, Integer> { @@ -818,7 +819,7 @@ public class JavaAPISuite implements Serializable { public void reduce() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); int sum = rdd.reduce(new AddInts()); - Assert.assertEquals(10, sum); + assertEquals(10, sum); } @Test @@ -830,21 +831,21 @@ public class JavaAPISuite implements Serializable { return v1 + v2; } }); - Assert.assertEquals(10.0, sum, 0.001); + assertEquals(10.0, sum, 0.001); } @Test public void fold() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); int sum = rdd.fold(0, new AddInts()); - Assert.assertEquals(10, sum); + assertEquals(10, sum); } @Test public void aggregate() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); int sum = rdd.aggregate(0, new AddInts(), new AddInts()); - Assert.assertEquals(10, sum); + assertEquals(10, sum); } @Test @@ -884,8 +885,8 @@ public class JavaAPISuite implements Serializable { return Arrays.asList(x.split(" ")).iterator(); } }); - Assert.assertEquals("Hello", words.first()); - Assert.assertEquals(11, words.count()); + assertEquals("Hello", words.first()); + assertEquals(11, words.count()); JavaPairRDD<String, String> pairsRDD = rdd.flatMapToPair( new PairFlatMapFunction<String, String, String>() { @@ -899,8 +900,8 @@ public class JavaAPISuite implements Serializable { } } ); - Assert.assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first()); - Assert.assertEquals(11, pairsRDD.count()); + assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first()); + assertEquals(11, pairsRDD.count()); JavaDoubleRDD doubles = rdd.flatMapToDouble(new DoubleFlatMapFunction<String>() { @Override @@ -912,8 +913,8 @@ public class JavaAPISuite implements Serializable { return lengths.iterator(); } }); - Assert.assertEquals(5.0, doubles.first(), 0.01); - Assert.assertEquals(11, pairsRDD.count()); + assertEquals(5.0, doubles.first(), 0.01); + assertEquals(11, pairsRDD.count()); } @SuppressWarnings("unchecked") @@ -959,7 +960,7 @@ public class JavaAPISuite implements Serializable { return Collections.singletonList(sum).iterator(); } }); - Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); + assertEquals("[3, 7]", partitionSums.collect().toString()); } @@ -977,7 +978,7 @@ public class JavaAPISuite implements Serializable { return Collections.singletonList(sum).iterator(); } }, false); - Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); + assertEquals("[3, 7]", partitionSums.collect().toString()); } @Test @@ -989,9 +990,9 @@ public class JavaAPISuite implements Serializable { new Tuple2<>("aa", 2), new Tuple2<>("aaa", 3) ), 2); - Assert.assertEquals(3, rdd1.getNumPartitions()); - Assert.assertEquals(2, rdd2.getNumPartitions()); - Assert.assertEquals(2, rdd3.getNumPartitions()); + assertEquals(3, rdd1.getNumPartitions()); + assertEquals(2, rdd2.getNumPartitions()); + assertEquals(2, rdd3.getNumPartitions()); } @Test @@ -1000,18 +1001,18 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> in1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 2); JavaRDD<Integer> repartitioned1 = in1.repartition(4); List<List<Integer>> result1 = repartitioned1.glom().collect(); - Assert.assertEquals(4, result1.size()); + assertEquals(4, result1.size()); for (List<Integer> l : result1) { - Assert.assertFalse(l.isEmpty()); + assertFalse(l.isEmpty()); } // Growing number of partitions JavaRDD<Integer> in2 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 4); JavaRDD<Integer> repartitioned2 = in2.repartition(2); List<List<Integer>> result2 = repartitioned2.glom().collect(); - Assert.assertEquals(2, result2.size()); + assertEquals(2, result2.size()); for (List<Integer> l: result2) { - Assert.assertFalse(l.isEmpty()); + assertFalse(l.isEmpty()); } } @@ -1020,7 +1021,7 @@ public class JavaAPISuite implements Serializable { public void persist() { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); doubleRDD = doubleRDD.persist(StorageLevel.DISK_ONLY()); - Assert.assertEquals(20, doubleRDD.sum(), 0.1); + assertEquals(20, doubleRDD.sum(), 0.1); List<Tuple2<Integer, String>> pairs = Arrays.asList( new Tuple2<>(1, "a"), @@ -1029,24 +1030,24 @@ public class JavaAPISuite implements Serializable { ); JavaPairRDD<Integer, String> pairRDD = sc.parallelizePairs(pairs); pairRDD = pairRDD.persist(StorageLevel.DISK_ONLY()); - Assert.assertEquals("a", pairRDD.first()._2()); + assertEquals("a", pairRDD.first()._2()); JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); rdd = rdd.persist(StorageLevel.DISK_ONLY()); - Assert.assertEquals(1, rdd.first().intValue()); + assertEquals(1, rdd.first().intValue()); } @Test public void iterator() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); TaskContext context = TaskContext$.MODULE$.empty(); - Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); + assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } @Test public void glom() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - Assert.assertEquals("[1, 2]", rdd.glom().first().toString()); + assertEquals("[1, 2]", rdd.glom().first().toString()); } // File input / output tests are largely adapted from FileSuite: @@ -1059,11 +1060,11 @@ public class JavaAPISuite implements Serializable { // Read the plain text file and check it's OK File outputFile = new File(outputDir, "part-00000"); String content = Files.toString(outputFile, StandardCharsets.UTF_8); - Assert.assertEquals("1\n2\n3\n4\n", content); + assertEquals("1\n2\n3\n4\n", content); // Also try reading it in as a text file RDD List<String> expected = Arrays.asList("1", "2", "3", "4"); JavaRDD<String> readRDD = sc.textFile(outputDir); - Assert.assertEquals(expected, readRDD.collect()); + assertEquals(expected, readRDD.collect()); } @Test @@ -1083,7 +1084,7 @@ public class JavaAPISuite implements Serializable { List<Tuple2<String, String>> result = readRDD.collect(); for (Tuple2<String, String> res : result) { - Assert.assertEquals(res._2(), container.get(new URI(res._1()).getPath())); + assertEquals(res._2(), container.get(new URI(res._1()).getPath())); } } @@ -1096,7 +1097,7 @@ public class JavaAPISuite implements Serializable { // Try reading it in as a text file RDD List<String> expected = Arrays.asList("1", "2", "3", "4"); JavaRDD<String> readRDD = sc.textFile(outputDir); - Assert.assertEquals(expected, readRDD.collect()); + assertEquals(expected, readRDD.collect()); } @SuppressWarnings("unchecked") @@ -1125,7 +1126,7 @@ public class JavaAPISuite implements Serializable { return new Tuple2<>(pair._1().get(), pair._2().toString()); } }); - Assert.assertEquals(pairs, readRDD.collect()); + assertEquals(pairs, readRDD.collect()); } @Test @@ -1145,7 +1146,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<String, PortableDataStream> readRDD = sc.binaryFiles(tempDirName, 3); List<Tuple2<String, PortableDataStream>> result = readRDD.collect(); for (Tuple2<String, PortableDataStream> res : result) { - Assert.assertArrayEquals(content1, res._2().toArray()); + assertArrayEquals(content1, res._2().toArray()); } } @@ -1174,7 +1175,7 @@ public class JavaAPISuite implements Serializable { List<Tuple2<String, PortableDataStream>> result = readRDD.collect(); for (Tuple2<String, PortableDataStream> res : result) { - Assert.assertArrayEquals(content1, res._2().toArray()); + assertArrayEquals(content1, res._2().toArray()); } } @@ -1197,10 +1198,10 @@ public class JavaAPISuite implements Serializable { channel1.close(); JavaRDD<byte[]> readRDD = sc.binaryRecords(tempDirName, content1.length); - Assert.assertEquals(numOfCopies,readRDD.count()); + assertEquals(numOfCopies,readRDD.count()); List<byte[]> result = readRDD.collect(); for (byte[] res : result) { - Assert.assertArrayEquals(content1, res); + assertArrayEquals(content1, res); } } @@ -1224,8 +1225,9 @@ public class JavaAPISuite implements Serializable { outputDir, IntWritable.class, Text.class, org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class); - JavaPairRDD<IntWritable, Text> output = sc.sequenceFile(outputDir, IntWritable.class, Text.class); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { + JavaPairRDD<IntWritable, Text> output = + sc.sequenceFile(outputDir, IntWritable.class, Text.class); + assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1254,7 +1256,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<IntWritable, Text> output = sc.newAPIHadoopFile(outputDir, org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class, IntWritable.class, Text.class, Job.getInstance().getConfiguration()); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { + assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1270,7 +1272,7 @@ public class JavaAPISuite implements Serializable { // Try reading the output back as an object file List<Integer> expected = Arrays.asList(1, 2, 3, 4); JavaRDD<Integer> readRDD = sc.objectFile(outputDir); - Assert.assertEquals(expected, readRDD.collect()); + assertEquals(expected, readRDD.collect()); } @SuppressWarnings("unchecked") @@ -1286,7 +1288,7 @@ public class JavaAPISuite implements Serializable { rdd.saveAsObjectFile(outputDir); // Try reading the output back as an object file JavaRDD<Tuple2<Integer, String>> readRDD = sc.objectFile(outputDir); - Assert.assertEquals(pairs, readRDD.collect()); + assertEquals(pairs, readRDD.collect()); } @SuppressWarnings("unchecked") @@ -1309,7 +1311,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir, SequenceFileInputFormat.class, IntWritable.class, Text.class); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { + assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1339,7 +1341,7 @@ public class JavaAPISuite implements Serializable { JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir, SequenceFileInputFormat.class, IntWritable.class, Text.class); - Assert.assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { + assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { @Override public String call(Tuple2<IntWritable, Text> x) { return x.toString(); @@ -1373,7 +1375,7 @@ public class JavaAPISuite implements Serializable { }; JavaRDD<Integer> sizes = rdd1.zipPartitions(rdd2, sizesFn); - Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); + assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); } @Test @@ -1387,7 +1389,7 @@ public class JavaAPISuite implements Serializable { intAccum.add(x); } }); - Assert.assertEquals((Integer) 25, intAccum.value()); + assertEquals((Integer) 25, intAccum.value()); final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0); rdd.foreach(new VoidFunction<Integer>() { @@ -1396,7 +1398,7 @@ public class JavaAPISuite implements Serializable { doubleAccum.add((double) x); } }); - Assert.assertEquals((Double) 25.0, doubleAccum.value()); + assertEquals((Double) 25.0, doubleAccum.value()); // Try a custom accumulator type AccumulatorParam<Float> floatAccumulatorParam = new AccumulatorParam<Float>() { @@ -1423,11 +1425,11 @@ public class JavaAPISuite implements Serializable { floatAccum.add((float) x); } }); - Assert.assertEquals((Float) 25.0f, floatAccum.value()); + assertEquals((Float) 25.0f, floatAccum.value()); // Test the setValue method floatAccum.setValue(5.0f); - Assert.assertEquals((Float) 5.0f, floatAccum.value()); + assertEquals((Float) 5.0f, floatAccum.value()); } @Test @@ -1439,33 +1441,33 @@ public class JavaAPISuite implements Serializable { return t.toString(); } }).collect(); - Assert.assertEquals(new Tuple2<>("1", 1), s.get(0)); - Assert.assertEquals(new Tuple2<>("2", 2), s.get(1)); + assertEquals(new Tuple2<>("1", 1), s.get(0)); + assertEquals(new Tuple2<>("2", 2), s.get(1)); } @Test public void checkpointAndComputation() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertFalse(rdd.isCheckpointed()); + assertFalse(rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertTrue(rdd.isCheckpointed()); - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect()); + assertTrue(rdd.isCheckpointed()); + assertEquals(Arrays.asList(1, 2, 3, 4, 5), rdd.collect()); } @Test public void checkpointAndRestore() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); sc.setCheckpointDir(tempDir.getAbsolutePath()); - Assert.assertFalse(rdd.isCheckpointed()); + assertFalse(rdd.isCheckpointed()); rdd.checkpoint(); rdd.count(); // Forces the DAG to cause a checkpoint - Assert.assertTrue(rdd.isCheckpointed()); + assertTrue(rdd.isCheckpointed()); - Assert.assertTrue(rdd.getCheckpointFile().isPresent()); + assertTrue(rdd.getCheckpointFile().isPresent()); JavaRDD<Integer> recovered = sc.checkpointFile(rdd.getCheckpointFile().get()); - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); + assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); } @Test @@ -1484,7 +1486,8 @@ public class JavaAPISuite implements Serializable { } }; - Function2<Integer, Integer, Integer> mergeValueFunction = new Function2<Integer, Integer, Integer>() { + Function2<Integer, Integer, Integer> mergeValueFunction = + new Function2<Integer, Integer, Integer>() { @Override public Integer call(Integer v1, Integer v2) { return v1 + v2; @@ -1495,7 +1498,7 @@ public class JavaAPISuite implements Serializable { .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction); Map<Integer, Integer> results = combinedRDD.collectAsMap(); ImmutableMap<Integer, Integer> expected = ImmutableMap.of(0, 9, 1, 5, 2, 7); - Assert.assertEquals(expected, results); + assertEquals(expected, results); Partitioner defaultPartitioner = Partitioner.defaultPartitioner( combinedRDD.rdd(), @@ -1510,7 +1513,7 @@ public class JavaAPISuite implements Serializable { false, new KryoSerializer(new SparkConf())); results = combinedRDD.collectAsMap(); - Assert.assertEquals(expected, results); + assertEquals(expected, results); } @SuppressWarnings("unchecked") @@ -1531,7 +1534,7 @@ public class JavaAPISuite implements Serializable { return new Tuple2<>(in._2(), in._1()); } }); - Assert.assertEquals(Arrays.asList( + assertEquals(Arrays.asList( new Tuple2<>(1, 1), new Tuple2<>(0, 2), new Tuple2<>(1, 3), @@ -1553,21 +1556,19 @@ public class JavaAPISuite implements Serializable { }); List<Integer>[] parts = rdd1.collectPartitions(new int[] {0}); - Assert.assertEquals(Arrays.asList(1, 2), parts[0]); + assertEquals(Arrays.asList(1, 2), parts[0]); parts = rdd1.collectPartitions(new int[] {1, 2}); - Assert.assertEquals(Arrays.asList(3, 4), parts[0]); - Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); + assertEquals(Arrays.asList(3, 4), parts[0]); + assertEquals(Arrays.asList(5, 6, 7), parts[1]); - Assert.assertEquals(Arrays.asList(new Tuple2<>(1, 1), + assertEquals(Arrays.asList(new Tuple2<>(1, 1), new Tuple2<>(2, 0)), rdd2.collectPartitions(new int[] {0})[0]); List<Tuple2<Integer,Integer>>[] parts2 = rdd2.collectPartitions(new int[] {1, 2}); - Assert.assertEquals(Arrays.asList(new Tuple2<>(3, 1), - new Tuple2<>(4, 0)), - parts2[0]); - Assert.assertEquals(Arrays.asList(new Tuple2<>(5, 1), + assertEquals(Arrays.asList(new Tuple2<>(3, 1), new Tuple2<>(4, 0)), parts2[0]); + assertEquals(Arrays.asList(new Tuple2<>(5, 1), new Tuple2<>(6, 0), new Tuple2<>(7, 1)), parts2[1]); @@ -1581,7 +1582,7 @@ public class JavaAPISuite implements Serializable { arrayData.add(i % size); } JavaRDD<Integer> simpleRdd = sc.parallelize(arrayData, 10); - Assert.assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1); + assertTrue(Math.abs((simpleRdd.countApproxDistinct(0.05) - size) / (size * 1.0)) <= 0.1); } @Test @@ -1599,7 +1600,7 @@ public class JavaAPISuite implements Serializable { double count = resItem._1(); long resCount = resItem._2(); double error = Math.abs((resCount - count) / count); - Assert.assertTrue(error < 0.1); + assertTrue(error < 0.1); } } @@ -1629,7 +1630,7 @@ public class JavaAPISuite implements Serializable { new ObjectOutputStream(bytes).writeObject(map); Map<String,Integer> deserializedMap = (Map<String,Integer>) new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject(); - Assert.assertEquals(1, deserializedMap.get("foo").intValue()); + assertEquals(1, deserializedMap.get("foo").intValue()); } @Test @@ -1648,14 +1649,14 @@ public class JavaAPISuite implements Serializable { fractions.put(1, 1.0); JavaPairRDD<Integer, Integer> wr = rdd2.sampleByKey(true, fractions, 1L); Map<Integer, Long> wrCounts = wr.countByKey(); - Assert.assertEquals(2, wrCounts.size()); - Assert.assertTrue(wrCounts.get(0) > 0); - Assert.assertTrue(wrCounts.get(1) > 0); + assertEquals(2, wrCounts.size()); + assertTrue(wrCounts.get(0) > 0); + assertTrue(wrCounts.get(1) > 0); JavaPairRDD<Integer, Integer> wor = rdd2.sampleByKey(false, fractions, 1L); Map<Integer, Long> worCounts = wor.countByKey(); - Assert.assertEquals(2, worCounts.size()); - Assert.assertTrue(worCounts.get(0) > 0); - Assert.assertTrue(worCounts.get(1) > 0); + assertEquals(2, worCounts.size()); + assertTrue(worCounts.get(0) > 0); + assertTrue(worCounts.get(1) > 0); } @Test @@ -1674,14 +1675,14 @@ public class JavaAPISuite implements Serializable { fractions.put(1, 1.0); JavaPairRDD<Integer, Integer> wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); Map<Integer, Long> wrExactCounts = wrExact.countByKey(); - Assert.assertEquals(2, wrExactCounts.size()); - Assert.assertTrue(wrExactCounts.get(0) == 2); - Assert.assertTrue(wrExactCounts.get(1) == 4); + assertEquals(2, wrExactCounts.size()); + assertTrue(wrExactCounts.get(0) == 2); + assertTrue(wrExactCounts.get(1) == 4); JavaPairRDD<Integer, Integer> worExact = rdd2.sampleByKeyExact(false, fractions, 1L); Map<Integer, Long> worExactCounts = worExact.countByKey(); - Assert.assertEquals(2, worExactCounts.size()); - Assert.assertTrue(worExactCounts.get(0) == 2); - Assert.assertTrue(worExactCounts.get(1) == 4); + assertEquals(2, worExactCounts.size()); + assertTrue(worExactCounts.get(0) == 2); + assertTrue(worExactCounts.get(1) == 4); } private static class SomeCustomClass implements Serializable { @@ -1697,8 +1698,9 @@ public class JavaAPISuite implements Serializable { data.add(new SomeCustomClass()); } JavaRDD<SomeCustomClass> rdd = sc.parallelize(data); - SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); - Assert.assertEquals(data.size(), collected.length); + SomeCustomClass[] collected = + (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); + assertEquals(data.size(), collected.length); } private static final class BuggyMapFunction<T> implements Function<T, T> { @@ -1715,10 +1717,10 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> rdd = sc.parallelize(data, 1); JavaFutureAction<List<Integer>> future = rdd.collectAsync(); List<Integer> result = future.get(); - Assert.assertEquals(data, result); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); + assertEquals(data, result); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); } @Test @@ -1727,11 +1729,11 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> rdd = sc.parallelize(data, 1); JavaFutureAction<List<Integer>> future = rdd.takeAsync(1); List<Integer> result = future.get(); - Assert.assertEquals(1, result.size()); - Assert.assertEquals((Integer) 1, result.get(0)); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); + assertEquals(1, result.size()); + assertEquals((Integer) 1, result.get(0)); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); } @Test @@ -1747,9 +1749,9 @@ public class JavaAPISuite implements Serializable { } ); future.get(); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); } @Test @@ -1758,10 +1760,10 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> rdd = sc.parallelize(data, 1); JavaFutureAction<Long> future = rdd.countAsync(); long count = future.get(); - Assert.assertEquals(data.size(), count); - Assert.assertFalse(future.isCancelled()); - Assert.assertTrue(future.isDone()); - Assert.assertEquals(1, future.jobIds().size()); + assertEquals(data.size(), count); + assertFalse(future.isCancelled()); + assertTrue(future.isDone()); + assertEquals(1, future.jobIds().size()); } @Test @@ -1775,11 +1777,11 @@ public class JavaAPISuite implements Serializable { } }); future.cancel(true); - Assert.assertTrue(future.isCancelled()); - Assert.assertTrue(future.isDone()); + assertTrue(future.isCancelled()); + assertTrue(future.isDone()); try { future.get(2000, TimeUnit.MILLISECONDS); - Assert.fail("Expected future.get() for cancelled job to throw CancellationException"); + fail("Expected future.get() for cancelled job to throw CancellationException"); } catch (CancellationException ignored) { // pass } @@ -1792,11 +1794,11 @@ public class JavaAPISuite implements Serializable { JavaFutureAction<Long> future = rdd.map(new BuggyMapFunction<Integer>()).countAsync(); try { future.get(2, TimeUnit.SECONDS); - Assert.fail("Expected future.get() for failed job to throw ExcecutionException"); + fail("Expected future.get() for failed job to throw ExcecutionException"); } catch (ExecutionException ee) { - Assert.assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!")); + assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!")); } - Assert.assertTrue(future.isDone()); + assertTrue(future.isDone()); } static class Class1 {} @@ -1806,7 +1808,7 @@ public class JavaAPISuite implements Serializable { public void testRegisterKryoClasses() { SparkConf conf = new SparkConf(); conf.registerKryoClasses(new Class<?>[]{ Class1.class, Class2.class }); - Assert.assertEquals( + assertEquals( Class1.class.getName() + "," + Class2.class.getName(), conf.get("spark.kryo.classesToRegister")); } @@ -1814,13 +1816,13 @@ public class JavaAPISuite implements Serializable { @Test public void testGetPersistentRDDs() { java.util.Map<Integer, JavaRDD<?>> cachedRddsMap = sc.getPersistentRDDs(); - Assert.assertTrue(cachedRddsMap.isEmpty()); + assertTrue(cachedRddsMap.isEmpty()); JavaRDD<String> rdd1 = sc.parallelize(Arrays.asList("a", "b")).setName("RDD1").cache(); JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("c", "d")).setName("RDD2").cache(); cachedRddsMap = sc.getPersistentRDDs(); - Assert.assertEquals(2, cachedRddsMap.size()); - Assert.assertEquals("RDD1", cachedRddsMap.get(0).name()); - Assert.assertEquals("RDD2", cachedRddsMap.get(1).name()); + assertEquals(2, cachedRddsMap.size()); + assertEquals("RDD1", cachedRddsMap.get(0).name()); + assertEquals("RDD2", cachedRddsMap.get(1).name()); } } diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 9aab2265c9..6667179b9d 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -92,9 +92,11 @@ public abstract class AbstractBytesToBytesMapSuite { spillFilesCreated.clear(); MockitoAnnotations.initMocks(this); when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); - when(diskBlockManager.createTempLocalBlock()).thenAnswer(new Answer<Tuple2<TempLocalBlockId, File>>() { + when(diskBlockManager.createTempLocalBlock()).thenAnswer( + new Answer<Tuple2<TempLocalBlockId, File>>() { @Override - public Tuple2<TempLocalBlockId, File> answer(InvocationOnMock invocationOnMock) throws Throwable { + public Tuple2<TempLocalBlockId, File> answer(InvocationOnMock invocationOnMock) + throws Throwable { TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); File file = File.createTempFile("spillFile", ".spill", tempDir); spillFilesCreated.add(file); @@ -544,7 +546,8 @@ public abstract class AbstractBytesToBytesMapSuite { @Test public void spillInIterator() throws IOException { - BytesToBytesMap map = new BytesToBytesMap(taskMemoryManager, blockManager, 1, 0.75, 1024, false); + BytesToBytesMap map = + new BytesToBytesMap(taskMemoryManager, blockManager, 1, 0.75, 1024, false); try { int i; for (i = 0; i < 1024; i++) { 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 a79ed58133..db50e551f2 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 @@ -103,9 +103,11 @@ public class UnsafeExternalSorterSuite { taskContext = mock(TaskContext.class); when(taskContext.taskMetrics()).thenReturn(new TaskMetrics()); when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); - when(diskBlockManager.createTempLocalBlock()).thenAnswer(new Answer<Tuple2<TempLocalBlockId, File>>() { + when(diskBlockManager.createTempLocalBlock()).thenAnswer( + new Answer<Tuple2<TempLocalBlockId, File>>() { @Override - public Tuple2<TempLocalBlockId, File> answer(InvocationOnMock invocationOnMock) throws Throwable { + public Tuple2<TempLocalBlockId, File> answer(InvocationOnMock invocationOnMock) + throws Throwable { TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); File file = File.createTempFile("spillFile", ".spill", tempDir); spillFilesCreated.add(file); 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 483319434d..f90214fffd 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 @@ -108,8 +108,8 @@ public class UnsafeInMemorySorterSuite { return (int) prefix1 - (int) prefix2; } }; - UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, memoryManager, recordComparator, - prefixComparator, dataToSort.length); + UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, memoryManager, + recordComparator, prefixComparator, dataToSort.length); // Given a page of records, insert those records into the sorter one-by-one: position = dataPage.getBaseOffset(); for (int i = 0; i < dataToSort.length; i++) { diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml index 9242be3d03..a1a88ac8cd 100644 --- a/dev/checkstyle-suppressions.xml +++ b/dev/checkstyle-suppressions.xml @@ -28,6 +28,12 @@ --> <suppressions> -<suppress checks=".*" - files="core/src/main/java/org/apache/spark/util/collection/TimSort.java"/> + <suppress checks=".*" + files="core/src/main/java/org/apache/spark/util/collection/TimSort.java"/> + <suppress checks=".*" + files="sql/core/src/main/java/org/apache/spark/sql/api.java/*"/> + <suppress checks="LineLength" + files="src/test/java/org/apache/spark/sql/hive/test/Complex.java"/> + <suppress checks="LineLength" + files="src/main/java/org/apache/spark/examples/JavaLogQuery.java"/> </suppressions> diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml index 2261cc95d4..b66dca9041 100644 --- a/dev/checkstyle.xml +++ b/dev/checkstyle.xml @@ -76,13 +76,10 @@ <property name="allowByTailComment" value="true"/> <property name="allowNonPrintableEscapes" value="true"/> </module> - <!-- TODO: 11/09/15 disabled - the lengths are currently > 100 in many places --> - <!-- <module name="LineLength"> <property name="max" value="100"/> <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/> </module> - --> <module name="NoLineWrap"/> <module name="EmptyBlock"> <property name="option" value="TEXT"/> @@ -167,5 +164,7 @@ </module> <module name="CommentsIndentation"/> <module name="UnusedImports"/> + <module name="RedundantImport"/> + <module name="RedundantModifier"/> </module> </module> diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index c3ef93c5b6..229d123441 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -84,13 +84,14 @@ public final class JavaPageRank { JavaRDD<String> lines = ctx.textFile(args[0], 1); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD<String, Iterable<String>> links = lines.mapToPair(new PairFunction<String, String, String>() { - @Override - public Tuple2<String, String> call(String s) { - String[] parts = SPACES.split(s); - return new Tuple2<>(parts[0], parts[1]); - } - }).distinct().groupByKey().cache(); + JavaPairRDD<String, Iterable<String>> links = lines.mapToPair( + new PairFunction<String, String, String>() { + @Override + public Tuple2<String, String> call(String s) { + String[] parts = SPACES.split(s); + return new Tuple2<>(parts[0], parts[1]); + } + }).distinct().groupByKey().cache(); // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. JavaPairRDD<String, Double> ranks = links.mapValues(new Function<Iterable<String>, Double>() { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 84dbea5caa..3ff5412b93 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -52,19 +52,21 @@ public final class JavaWordCount { } }); - JavaPairRDD<String, Integer> ones = words.mapToPair(new PairFunction<String, String, Integer>() { - @Override - public Tuple2<String, Integer> call(String s) { - return new Tuple2<>(s, 1); - } - }); + JavaPairRDD<String, Integer> ones = words.mapToPair( + new PairFunction<String, String, Integer>() { + @Override + public Tuple2<String, Integer> call(String s) { + return new Tuple2<>(s, 1); + } + }); - JavaPairRDD<String, Integer> counts = ones.reduceByKey(new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }); + JavaPairRDD<String, Integer> counts = ones.reduceByKey( + new Function2<Integer, Integer, Integer>() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); List<Tuple2<String, Integer>> output = counts.collect(); for (Tuple2<?,?> tuple : output) { diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java index 5bd61fe508..8214952f80 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDecisionTreeClassificationExample.java @@ -39,7 +39,10 @@ public class JavaDecisionTreeClassificationExample { // $example on$ // Load the data stored in LIBSVM format as a DataFrame. - Dataset<Row> data = sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset<Row> data = sqlContext + .read() + .format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 8a10dd48aa..fbd8817669 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -107,11 +107,11 @@ public class JavaDeveloperApiExample { class MyJavaLogisticRegression extends Classifier<Vector, MyJavaLogisticRegression, MyJavaLogisticRegressionModel> { - public MyJavaLogisticRegression() { + MyJavaLogisticRegression() { init(); } - public MyJavaLogisticRegression(String uid) { + MyJavaLogisticRegression(String uid) { this.uid_ = uid; init(); } @@ -177,7 +177,7 @@ class MyJavaLogisticRegressionModel private Vector coefficients_; public Vector coefficients() { return coefficients_; } - public MyJavaLogisticRegressionModel(String uid, Vector coefficients) { + MyJavaLogisticRegressionModel(String uid, Vector coefficients) { this.uid_ = uid; this.coefficients_ = coefficients; } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java index c2cb955385..553070dace 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaGradientBoostedTreeClassifierExample.java @@ -40,7 +40,8 @@ public class JavaGradientBoostedTreeClassifierExample { // $example on$ // Load and parse the data file, converting it to a DataFrame. - Dataset<Row> data = sqlContext.read().format("libsvm").load("data/mllib/sample_libsvm_data.txt"); + Dataset<Row> data = sqlContext.read().format("libsvm") + .load("data/mllib/sample_libsvm_data.txt"); // Index labels, adding metadata to the label column. // Fit on whole dataset to include all labels in index. diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java index 3d8babba04..7561a1f653 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaBinaryClassificationMetricsExample.java @@ -65,7 +65,8 @@ public class JavaBinaryClassificationMetricsExample { ); // Get evaluation metrics. - BinaryClassificationMetrics metrics = new BinaryClassificationMetrics(predictionAndLabels.rdd()); + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(predictionAndLabels.rdd()); // Precision by threshold JavaRDD<Tuple2<Object, Object>> precision = metrics.precisionByThreshold().toJavaRDD(); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java index 0e15f75508..c6361a3729 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaIsotonicRegressionExample.java @@ -48,7 +48,8 @@ public class JavaIsotonicRegressionExample { ); // Split data into training (60%) and test (40%) sets. - JavaRDD<Tuple3<Double, Double, Double>>[] splits = parsedData.randomSplit(new double[]{0.6, 0.4}, 11L); + JavaRDD<Tuple3<Double, Double, Double>>[] splits = + parsedData.randomSplit(new double[]{0.6, 0.4}, 11L); JavaRDD<Tuple3<Double, Double, Double>> training = splits[0]; JavaRDD<Tuple3<Double, Double, Double>> test = splits[1]; @@ -80,7 +81,8 @@ public class JavaIsotonicRegressionExample { // Save and load model model.save(jsc.sc(), "target/tmp/myIsotonicRegressionModel"); - IsotonicRegressionModel sameModel = IsotonicRegressionModel.load(jsc.sc(), "target/tmp/myIsotonicRegressionModel"); + IsotonicRegressionModel sameModel = + IsotonicRegressionModel.load(jsc.sc(), "target/tmp/myIsotonicRegressionModel"); // $example off$ jsc.stop(); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java index 4c8755916c..984909cb94 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaStreamingTestExample.java @@ -18,7 +18,6 @@ package org.apache.spark.examples.mllib; -import org.apache.spark.Accumulator; import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java index bfbad91e4f..769b21cecf 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaDirectKafkaWordCount.java @@ -40,7 +40,8 @@ import org.apache.spark.streaming.Durations; * <topics> is a list of one or more kafka topics to consume from * * Example: - * $ bin/run-example streaming.JavaDirectKafkaWordCount broker1-host:port,broker2-host:port topic1,topic2 + * $ bin/run-example streaming.JavaDirectKafkaWordCount broker1-host:port,broker2-host:port \ + * topic1,topic2 */ public final class JavaDirectKafkaWordCount { diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java index 426eaa5f0a..62413b4606 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java @@ -30,7 +30,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java index a597ecbc5b..e5fb2bfbfa 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaRecoverableNetworkWordCount.java @@ -155,9 +155,11 @@ public final class JavaRecoverableNetworkWordCount { @Override public void call(JavaPairRDD<String, Integer> rdd, Time time) throws IOException { // Get or register the blacklist Broadcast - final Broadcast<List<String>> blacklist = JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); + final Broadcast<List<String>> blacklist = + JavaWordBlacklist.getInstance(new JavaSparkContext(rdd.context())); // Get or register the droppedWordsCounter Accumulator - final Accumulator<Integer> droppedWordsCounter = JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); + final Accumulator<Integer> droppedWordsCounter = + JavaDroppedWordsCounter.getInstance(new JavaSparkContext(rdd.context())); // Use blacklist to drop words and use droppedWordsCounter to count them String counts = rdd.filter(new Function<Tuple2<String, Integer>, Boolean>() { @Override @@ -210,7 +212,8 @@ public final class JavaRecoverableNetworkWordCount { } }; - JavaStreamingContext ssc = JavaStreamingContext.getOrCreate(checkpointDirectory, createContextFunc); + JavaStreamingContext ssc = + JavaStreamingContext.getOrCreate(checkpointDirectory, createContextFunc); ssc.start(); ssc.awaitTermination(); } diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java index 6beab90f08..4230dab52e 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaStatefulNetworkWordCount.java @@ -91,7 +91,8 @@ public class JavaStatefulNetworkWordCount { Function3<String, Optional<Integer>, State<Integer>, Tuple2<String, Integer>> mappingFunc = new Function3<String, Optional<Integer>, State<Integer>, Tuple2<String, Integer>>() { @Override - public Tuple2<String, Integer> call(String word, Optional<Integer> one, State<Integer> state) { + public Tuple2<String, Integer> call(String word, Optional<Integer> one, + State<Integer> state) { int sum = one.orElse(0) + (state.exists() ? state.get() : 0); Tuple2<String, Integer> output = new Tuple2<>(word, sum); state.update(sum); diff --git a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 5dc825dfdc..0e43e9272d 100644 --- a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -140,7 +140,8 @@ public final class JavaKinesisWordCountASL { // needs to be public for access fr for (int i = 0; i < numStreams; i++) { streamsList.add( KinesisUtils.createStream(jssc, kinesisAppName, streamName, endpointUrl, regionName, - InitialPositionInStream.LATEST, kinesisCheckpointInterval, StorageLevel.MEMORY_AND_DISK_2()) + InitialPositionInStream.LATEST, kinesisCheckpointInterval, + StorageLevel.MEMORY_AND_DISK_2()) ); } @@ -167,7 +168,7 @@ public final class JavaKinesisWordCountASL { // needs to be public for access fr new PairFunction<String, String, Integer>() { @Override public Tuple2<String, Integer> call(String s) { - return new Tuple2<String, Integer>(s, 1); + return new Tuple2<>(s, 1); } } ).reduceByKey( diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index f6c7e07654..587fda7a3c 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -57,7 +57,7 @@ abstract class AbstractCommandBuilder { // properties files multiple times. private Map<String, String> effectiveConfig; - public AbstractCommandBuilder() { + AbstractCommandBuilder() { this.appArgs = new ArrayList<>(); this.childEnv = new HashMap<>(); this.conf = new HashMap<>(); diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index 37afafea28..39fdf300e2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -32,7 +32,7 @@ class CommandBuilderUtils { static final String ENV_SPARK_HOME = "SPARK_HOME"; /** The set of known JVM vendors. */ - static enum JavaVendor { + enum JavaVendor { Oracle, IBM, OpenJDK, Unknown }; diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java index e9caf0b3cb..625d026321 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkAppHandle.java @@ -32,7 +32,7 @@ public interface SparkAppHandle { * * @since 1.6.0 */ - public enum State { + enum State { /** The application has not reported back yet. */ UNKNOWN(false), /** The application has connected to the handle. */ diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index 536f0dc58f..e160a5a47e 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -18,7 +18,6 @@ package org.apache.spark.ml.classification; import java.io.Serializable; -import java.lang.Math; import java.util.List; import org.junit.After; diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java index d493a7fcec..00f4476841 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaOneVsRestSuite.java @@ -48,7 +48,8 @@ public class JavaOneVsRestSuite implements Serializable { jsql = new SQLContext(jsc); int nPoints = 3; - // The following coefficients and xMean/xVariance are computed from iris dataset with lambda=0.2. + // The following coefficients and xMean/xVariance are computed from iris dataset with + // lambda=0.2. // As a result, we are drawing samples from probability distribution of an actual model. double[] coefficients = { -0.57997, 0.912083, -0.371077, -0.819866, 2.688191, diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java index 5812037dee..bdcbde5e26 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaStopWordsRemoverSuite.java @@ -63,7 +63,8 @@ public class JavaStopWordsRemoverSuite { RowFactory.create(Arrays.asList("Mary", "had", "a", "little", "lamb")) ); StructType schema = new StructType(new StructField[] { - new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) + new StructField("raw", DataTypes.createArrayType(DataTypes.StringType), false, + Metadata.empty()) }); Dataset<Row> dataset = jsql.createDataFrame(data, schema); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 0ad0f4976c..d85147e961 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -171,7 +171,7 @@ public final class UnsafeExternalRowSorter { private final UnsafeRow row1; private final UnsafeRow row2; - public RowComparator(Ordering<InternalRow> ordering, int numFields) { + RowComparator(Ordering<InternalRow> ordering, int numFields) { this.numFields = numFields; this.row1 = new UnsafeRow(numFields); this.row2 = new UnsafeRow(numFields); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java index 57e8218f3b..acf6c583bb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java @@ -236,8 +236,8 @@ public final class UnsafeFixedWidthAggregationMap { /** * Sorts the map's records in place, spill them to disk, and returns an [[UnsafeKVExternalSorter]] * - * Note that the map will be reset for inserting new records, and the returned sorter can NOT be used - * to insert records. + * Note that the map will be reset for inserting new records, and the returned sorter can NOT be + * used to insert records. */ public UnsafeKVExternalSorter destructAndCreateExternalSorter() throws IOException { return new UnsafeKVExternalSorter( diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 51e10b0e93..9e08675c3e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -198,7 +198,7 @@ public final class UnsafeKVExternalSorter { private final UnsafeRow row2; private final int numKeyFields; - public KVComparator(BaseOrdering ordering, int numKeyFields) { + KVComparator(BaseOrdering ordering, int numKeyFields) { this.numKeyFields = numKeyFields; this.row1 = new UnsafeRow(numKeyFields); this.row2 = new UnsafeRow(numKeyFields); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java index ffcc9c2ace..04adf1fb6d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVector.java @@ -94,7 +94,7 @@ public abstract class ColumnVector { } @Override - public final int numElements() { return length; } + public int numElements() { return length; } @Override public ArrayData copy() { @@ -175,10 +175,10 @@ public abstract class ColumnVector { } @Override - public final boolean isNullAt(int ordinal) { return data.getIsNull(offset + ordinal); } + public boolean isNullAt(int ordinal) { return data.getIsNull(offset + ordinal); } @Override - public final boolean getBoolean(int ordinal) { + public boolean getBoolean(int ordinal) { throw new NotImplementedException(); } 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 c462ab1a13..b6fa9a0b9e 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 @@ -115,20 +115,20 @@ public final class ColumnarBatch { * Marks this row as being filtered out. This means a subsequent iteration over the rows * in this batch will not include this row. */ - public final void markFiltered() { + public void markFiltered() { parent.markFiltered(rowId); } public ColumnVector[] columns() { return columns; } @Override - public final int numFields() { return columns.length; } + public int numFields() { return columns.length; } @Override /** * Revisit this. This is expensive. This is currently only used in test paths. */ - public final InternalRow copy() { + public InternalRow copy() { GenericMutableRow row = new GenericMutableRow(columns.length); for (int i = 0; i < numFields(); i++) { if (isNullAt(i)) { @@ -163,73 +163,73 @@ public final class ColumnarBatch { } @Override - public final boolean anyNull() { + public boolean anyNull() { throw new NotImplementedException(); } @Override - public final boolean isNullAt(int ordinal) { return columns[ordinal].getIsNull(rowId); } + public boolean isNullAt(int ordinal) { return columns[ordinal].getIsNull(rowId); } @Override - public final boolean getBoolean(int ordinal) { return columns[ordinal].getBoolean(rowId); } + public boolean getBoolean(int ordinal) { return columns[ordinal].getBoolean(rowId); } @Override - public final byte getByte(int ordinal) { return columns[ordinal].getByte(rowId); } + public byte getByte(int ordinal) { return columns[ordinal].getByte(rowId); } @Override - public final short getShort(int ordinal) { return columns[ordinal].getShort(rowId); } + public short getShort(int ordinal) { return columns[ordinal].getShort(rowId); } @Override - public final int getInt(int ordinal) { return columns[ordinal].getInt(rowId); } + public int getInt(int ordinal) { return columns[ordinal].getInt(rowId); } @Override - public final long getLong(int ordinal) { return columns[ordinal].getLong(rowId); } + public long getLong(int ordinal) { return columns[ordinal].getLong(rowId); } @Override - public final float getFloat(int ordinal) { return columns[ordinal].getFloat(rowId); } + public float getFloat(int ordinal) { return columns[ordinal].getFloat(rowId); } @Override - public final double getDouble(int ordinal) { return columns[ordinal].getDouble(rowId); } + public double getDouble(int ordinal) { return columns[ordinal].getDouble(rowId); } @Override - public final Decimal getDecimal(int ordinal, int precision, int scale) { + public Decimal getDecimal(int ordinal, int precision, int scale) { return columns[ordinal].getDecimal(rowId, precision, scale); } @Override - public final UTF8String getUTF8String(int ordinal) { + public UTF8String getUTF8String(int ordinal) { return columns[ordinal].getUTF8String(rowId); } @Override - public final byte[] getBinary(int ordinal) { + public byte[] getBinary(int ordinal) { return columns[ordinal].getBinary(rowId); } @Override - public final CalendarInterval getInterval(int ordinal) { + public CalendarInterval getInterval(int ordinal) { final int months = columns[ordinal].getChildColumn(0).getInt(rowId); final long microseconds = columns[ordinal].getChildColumn(1).getLong(rowId); return new CalendarInterval(months, microseconds); } @Override - public final InternalRow getStruct(int ordinal, int numFields) { + public InternalRow getStruct(int ordinal, int numFields) { return columns[ordinal].getStruct(rowId); } @Override - public final ArrayData getArray(int ordinal) { + public ArrayData getArray(int ordinal) { return columns[ordinal].getArray(rowId); } @Override - public final MapData getMap(int ordinal) { + public MapData getMap(int ordinal) { throw new NotImplementedException(); } @Override - public final Object get(int ordinal, DataType dataType) { + public Object get(int ordinal, DataType dataType) { throw new NotImplementedException(); } } @@ -357,7 +357,7 @@ public final class ColumnarBatch { * Marks this row as being filtered out. This means a subsequent iteration over the rows * in this batch will not include this row. */ - public final void markFiltered(int rowId) { + public void markFiltered(int rowId) { assert(!filteredRows[rowId]); filteredRows[rowId] = true; ++numRowsFiltered; @@ -367,7 +367,7 @@ public final class ColumnarBatch { * Marks a given column as non-nullable. Any row that has a NULL value for the corresponding * attribute is filtered out. */ - public final void filterNullsInColumn(int ordinal) { + public void filterNullsInColumn(int ordinal) { nullFilteredColumns.add(ordinal); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index b06b7f2457..d5a9163274 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -52,7 +52,7 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final long valuesNativeAddress() { + public long valuesNativeAddress() { return data; } @@ -62,7 +62,7 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void close() { + public void close() { Platform.freeMemory(nulls); Platform.freeMemory(data); Platform.freeMemory(lengthData); @@ -78,19 +78,19 @@ public final class OffHeapColumnVector extends ColumnVector { // @Override - public final void putNotNull(int rowId) { + public void putNotNull(int rowId) { Platform.putByte(null, nulls + rowId, (byte) 0); } @Override - public final void putNull(int rowId) { + public void putNull(int rowId) { Platform.putByte(null, nulls + rowId, (byte) 1); ++numNulls; anyNullsSet = true; } @Override - public final void putNulls(int rowId, int count) { + public void putNulls(int rowId, int count) { long offset = nulls + rowId; for (int i = 0; i < count; ++i, ++offset) { Platform.putByte(null, offset, (byte) 1); @@ -100,7 +100,7 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void putNotNulls(int rowId, int count) { + public void putNotNulls(int rowId, int count) { if (!anyNullsSet) return; long offset = nulls + rowId; for (int i = 0; i < count; ++i, ++offset) { @@ -109,7 +109,7 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final boolean getIsNull(int rowId) { + public boolean getIsNull(int rowId) { return Platform.getByte(null, nulls + rowId) == 1; } @@ -118,12 +118,12 @@ public final class OffHeapColumnVector extends ColumnVector { // @Override - public final void putBoolean(int rowId, boolean value) { + public void putBoolean(int rowId, boolean value) { Platform.putByte(null, data + rowId, (byte)((value) ? 1 : 0)); } @Override - public final void putBooleans(int rowId, int count, boolean value) { + public void putBooleans(int rowId, int count, boolean value) { byte v = (byte)((value) ? 1 : 0); for (int i = 0; i < count; ++i) { Platform.putByte(null, data + rowId + i, v); @@ -131,32 +131,32 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final boolean getBoolean(int rowId) { return Platform.getByte(null, data + rowId) == 1; } + public boolean getBoolean(int rowId) { return Platform.getByte(null, data + rowId) == 1; } // // APIs dealing with Bytes // @Override - public final void putByte(int rowId, byte value) { + public void putByte(int rowId, byte value) { Platform.putByte(null, data + rowId, value); } @Override - public final void putBytes(int rowId, int count, byte value) { + public void putBytes(int rowId, int count, byte value) { for (int i = 0; i < count; ++i) { Platform.putByte(null, data + rowId + i, value); } } @Override - public final void putBytes(int rowId, int count, byte[] src, int srcIndex) { + public void putBytes(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId, count); } @Override - public final byte getByte(int rowId) { + public byte getByte(int rowId) { if (dictionary == null) { return Platform.getByte(null, data + rowId); } else { @@ -169,12 +169,12 @@ public final class OffHeapColumnVector extends ColumnVector { // @Override - public final void putShort(int rowId, short value) { + public void putShort(int rowId, short value) { Platform.putShort(null, data + 2 * rowId, value); } @Override - public final void putShorts(int rowId, int count, short value) { + public void putShorts(int rowId, int count, short value) { long offset = data + 2 * rowId; for (int i = 0; i < count; ++i, offset += 4) { Platform.putShort(null, offset, value); @@ -182,13 +182,13 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void putShorts(int rowId, int count, short[] src, int srcIndex) { + public void putShorts(int rowId, int count, short[] src, int srcIndex) { Platform.copyMemory(src, Platform.SHORT_ARRAY_OFFSET + srcIndex * 2, null, data + 2 * rowId, count * 2); } @Override - public final short getShort(int rowId) { + public short getShort(int rowId) { if (dictionary == null) { return Platform.getShort(null, data + 2 * rowId); } else { @@ -201,12 +201,12 @@ public final class OffHeapColumnVector extends ColumnVector { // @Override - public final void putInt(int rowId, int value) { + public void putInt(int rowId, int value) { Platform.putInt(null, data + 4 * rowId, value); } @Override - public final void putInts(int rowId, int count, int value) { + public void putInts(int rowId, int count, int value) { long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { Platform.putInt(null, offset, value); @@ -214,19 +214,19 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void putInts(int rowId, int count, int[] src, int srcIndex) { + public void putInts(int rowId, int count, int[] src, int srcIndex) { Platform.copyMemory(src, Platform.INT_ARRAY_OFFSET + srcIndex * 4, null, data + 4 * rowId, count * 4); } @Override - public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, null, data + 4 * rowId, count * 4); } @Override - public final int getInt(int rowId) { + public int getInt(int rowId) { if (dictionary == null) { return Platform.getInt(null, data + 4 * rowId); } else { @@ -239,12 +239,12 @@ public final class OffHeapColumnVector extends ColumnVector { // @Override - public final void putLong(int rowId, long value) { + public void putLong(int rowId, long value) { Platform.putLong(null, data + 8 * rowId, value); } @Override - public final void putLongs(int rowId, int count, long value) { + public void putLongs(int rowId, int count, long value) { long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { Platform.putLong(null, offset, value); @@ -252,19 +252,19 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void putLongs(int rowId, int count, long[] src, int srcIndex) { + public void putLongs(int rowId, int count, long[] src, int srcIndex) { Platform.copyMemory(src, Platform.LONG_ARRAY_OFFSET + srcIndex * 8, null, data + 8 * rowId, count * 8); } @Override - public final void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, srcIndex + Platform.BYTE_ARRAY_OFFSET, null, data + 8 * rowId, count * 8); } @Override - public final long getLong(int rowId) { + public long getLong(int rowId) { if (dictionary == null) { return Platform.getLong(null, data + 8 * rowId); } else { @@ -277,12 +277,12 @@ public final class OffHeapColumnVector extends ColumnVector { // @Override - public final void putFloat(int rowId, float value) { + public void putFloat(int rowId, float value) { Platform.putFloat(null, data + rowId * 4, value); } @Override - public final void putFloats(int rowId, int count, float value) { + public void putFloats(int rowId, int count, float value) { long offset = data + 4 * rowId; for (int i = 0; i < count; ++i, offset += 4) { Platform.putFloat(null, offset, value); @@ -290,19 +290,19 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void putFloats(int rowId, int count, float[] src, int srcIndex) { + public void putFloats(int rowId, int count, float[] src, int srcIndex) { Platform.copyMemory(src, Platform.FLOAT_ARRAY_OFFSET + srcIndex * 4, null, data + 4 * rowId, count * 4); } @Override - public final void putFloats(int rowId, int count, byte[] src, int srcIndex) { + public void putFloats(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId * 4, count * 4); } @Override - public final float getFloat(int rowId) { + public float getFloat(int rowId) { if (dictionary == null) { return Platform.getFloat(null, data + rowId * 4); } else { @@ -316,12 +316,12 @@ public final class OffHeapColumnVector extends ColumnVector { // @Override - public final void putDouble(int rowId, double value) { + public void putDouble(int rowId, double value) { Platform.putDouble(null, data + rowId * 8, value); } @Override - public final void putDoubles(int rowId, int count, double value) { + public void putDoubles(int rowId, int count, double value) { long offset = data + 8 * rowId; for (int i = 0; i < count; ++i, offset += 8) { Platform.putDouble(null, offset, value); @@ -329,19 +329,19 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + public void putDoubles(int rowId, int count, double[] src, int srcIndex) { Platform.copyMemory(src, Platform.DOUBLE_ARRAY_OFFSET + srcIndex * 8, null, data + 8 * rowId, count * 8); } @Override - public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId * 8, count * 8); } @Override - public final double getDouble(int rowId) { + public double getDouble(int rowId) { if (dictionary == null) { return Platform.getDouble(null, data + rowId * 8); } else { @@ -353,25 +353,25 @@ public final class OffHeapColumnVector extends ColumnVector { // APIs dealing with Arrays. // @Override - public final void putArray(int rowId, int offset, int length) { + public void putArray(int rowId, int offset, int length) { assert(offset >= 0 && offset + length <= childColumns[0].capacity); Platform.putInt(null, lengthData + 4 * rowId, length); Platform.putInt(null, offsetData + 4 * rowId, offset); } @Override - public final int getArrayLength(int rowId) { + public int getArrayLength(int rowId) { return Platform.getInt(null, lengthData + 4 * rowId); } @Override - public final int getArrayOffset(int rowId) { + public int getArrayOffset(int rowId) { return Platform.getInt(null, offsetData + 4 * rowId); } // APIs dealing with ByteArrays @Override - public final int putByteArray(int rowId, byte[] value, int offset, int length) { + public int putByteArray(int rowId, byte[] value, int offset, int length) { int result = arrayData().appendBytes(length, value, offset); Platform.putInt(null, lengthData + 4 * rowId, length); Platform.putInt(null, offsetData + 4 * rowId, result); @@ -379,7 +379,7 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void loadBytes(ColumnVector.Array array) { + public void loadBytes(ColumnVector.Array array) { if (array.tmpByteArray.length < array.length) array.tmpByteArray = new byte[array.length]; Platform.copyMemory( null, data + array.offset, array.tmpByteArray, Platform.BYTE_ARRAY_OFFSET, array.length); @@ -388,12 +388,12 @@ public final class OffHeapColumnVector extends ColumnVector { } @Override - public final void reserve(int requiredCapacity) { + public void reserve(int requiredCapacity) { if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); } // Split out the slow path. - private final void reserveInternal(int newCapacity) { + private void reserveInternal(int newCapacity) { if (this.resultArray != null) { this.lengthData = Platform.reallocateMemory(lengthData, elementsAppended * 4, newCapacity * 4); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index 03160d1ec3..5b671a7432 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -52,16 +52,16 @@ public final class OnHeapColumnVector extends ColumnVector { } @Override - public final long valuesNativeAddress() { + public long valuesNativeAddress() { throw new RuntimeException("Cannot get native address for on heap column"); } @Override - public final long nullsNativeAddress() { + public long nullsNativeAddress() { throw new RuntimeException("Cannot get native address for on heap column"); } @Override - public final void close() { + public void close() { } // @@ -69,19 +69,19 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putNotNull(int rowId) { + public void putNotNull(int rowId) { nulls[rowId] = (byte)0; } @Override - public final void putNull(int rowId) { + public void putNull(int rowId) { nulls[rowId] = (byte)1; ++numNulls; anyNullsSet = true; } @Override - public final void putNulls(int rowId, int count) { + public void putNulls(int rowId, int count) { for (int i = 0; i < count; ++i) { nulls[rowId + i] = (byte)1; } @@ -90,7 +90,7 @@ public final class OnHeapColumnVector extends ColumnVector { } @Override - public final void putNotNulls(int rowId, int count) { + public void putNotNulls(int rowId, int count) { if (!anyNullsSet) return; for (int i = 0; i < count; ++i) { nulls[rowId + i] = (byte)0; @@ -98,7 +98,7 @@ public final class OnHeapColumnVector extends ColumnVector { } @Override - public final boolean getIsNull(int rowId) { + public boolean getIsNull(int rowId) { return nulls[rowId] == 1; } @@ -107,12 +107,12 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putBoolean(int rowId, boolean value) { + public void putBoolean(int rowId, boolean value) { byteData[rowId] = (byte)((value) ? 1 : 0); } @Override - public final void putBooleans(int rowId, int count, boolean value) { + public void putBooleans(int rowId, int count, boolean value) { byte v = (byte)((value) ? 1 : 0); for (int i = 0; i < count; ++i) { byteData[i + rowId] = v; @@ -120,7 +120,7 @@ public final class OnHeapColumnVector extends ColumnVector { } @Override - public final boolean getBoolean(int rowId) { + public boolean getBoolean(int rowId) { return byteData[rowId] == 1; } @@ -131,24 +131,24 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putByte(int rowId, byte value) { + public void putByte(int rowId, byte value) { byteData[rowId] = value; } @Override - public final void putBytes(int rowId, int count, byte value) { + public void putBytes(int rowId, int count, byte value) { for (int i = 0; i < count; ++i) { byteData[i + rowId] = value; } } @Override - public final void putBytes(int rowId, int count, byte[] src, int srcIndex) { + public void putBytes(int rowId, int count, byte[] src, int srcIndex) { System.arraycopy(src, srcIndex, byteData, rowId, count); } @Override - public final byte getByte(int rowId) { + public byte getByte(int rowId) { if (dictionary == null) { return byteData[rowId]; } else { @@ -161,24 +161,24 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putShort(int rowId, short value) { + public void putShort(int rowId, short value) { shortData[rowId] = value; } @Override - public final void putShorts(int rowId, int count, short value) { + public void putShorts(int rowId, int count, short value) { for (int i = 0; i < count; ++i) { shortData[i + rowId] = value; } } @Override - public final void putShorts(int rowId, int count, short[] src, int srcIndex) { + public void putShorts(int rowId, int count, short[] src, int srcIndex) { System.arraycopy(src, srcIndex, shortData, rowId, count); } @Override - public final short getShort(int rowId) { + public short getShort(int rowId) { if (dictionary == null) { return shortData[rowId]; } else { @@ -192,24 +192,24 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putInt(int rowId, int value) { + public void putInt(int rowId, int value) { intData[rowId] = value; } @Override - public final void putInts(int rowId, int count, int value) { + public void putInts(int rowId, int count, int value) { for (int i = 0; i < count; ++i) { intData[i + rowId] = value; } } @Override - public final void putInts(int rowId, int count, int[] src, int srcIndex) { + public void putInts(int rowId, int count, int[] src, int srcIndex) { System.arraycopy(src, srcIndex, intData, rowId, count); } @Override - public final void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; for (int i = 0; i < count; ++i) { intData[i + rowId] = Platform.getInt(src, srcOffset);; @@ -219,7 +219,7 @@ public final class OnHeapColumnVector extends ColumnVector { } @Override - public final int getInt(int rowId) { + public int getInt(int rowId) { if (dictionary == null) { return intData[rowId]; } else { @@ -232,24 +232,24 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putLong(int rowId, long value) { + public void putLong(int rowId, long value) { longData[rowId] = value; } @Override - public final void putLongs(int rowId, int count, long value) { + public void putLongs(int rowId, int count, long value) { for (int i = 0; i < count; ++i) { longData[i + rowId] = value; } } @Override - public final void putLongs(int rowId, int count, long[] src, int srcIndex) { + public void putLongs(int rowId, int count, long[] src, int srcIndex) { System.arraycopy(src, srcIndex, longData, rowId, count); } @Override - public final void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; for (int i = 0; i < count; ++i) { longData[i + rowId] = Platform.getLong(src, srcOffset); @@ -259,7 +259,7 @@ public final class OnHeapColumnVector extends ColumnVector { } @Override - public final long getLong(int rowId) { + public long getLong(int rowId) { if (dictionary == null) { return longData[rowId]; } else { @@ -272,26 +272,26 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putFloat(int rowId, float value) { floatData[rowId] = value; } + public void putFloat(int rowId, float value) { floatData[rowId] = value; } @Override - public final void putFloats(int rowId, int count, float value) { + public void putFloats(int rowId, int count, float value) { Arrays.fill(floatData, rowId, rowId + count, value); } @Override - public final void putFloats(int rowId, int count, float[] src, int srcIndex) { + public void putFloats(int rowId, int count, float[] src, int srcIndex) { System.arraycopy(src, srcIndex, floatData, rowId, count); } @Override - public final void putFloats(int rowId, int count, byte[] src, int srcIndex) { + public void putFloats(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, floatData, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4, count * 4); } @Override - public final float getFloat(int rowId) { + public float getFloat(int rowId) { if (dictionary == null) { return floatData[rowId]; } else { @@ -304,28 +304,28 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final void putDouble(int rowId, double value) { + public void putDouble(int rowId, double value) { doubleData[rowId] = value; } @Override - public final void putDoubles(int rowId, int count, double value) { + public void putDoubles(int rowId, int count, double value) { Arrays.fill(doubleData, rowId, rowId + count, value); } @Override - public final void putDoubles(int rowId, int count, double[] src, int srcIndex) { + public void putDoubles(int rowId, int count, double[] src, int srcIndex) { System.arraycopy(src, srcIndex, doubleData, rowId, count); } @Override - public final void putDoubles(int rowId, int count, byte[] src, int srcIndex) { + public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, Platform.DOUBLE_ARRAY_OFFSET + rowId * 8, count * 8); } @Override - public final double getDouble(int rowId) { + public double getDouble(int rowId) { if (dictionary == null) { return doubleData[rowId]; } else { @@ -338,22 +338,22 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final int getArrayLength(int rowId) { + public int getArrayLength(int rowId) { return arrayLengths[rowId]; } @Override - public final int getArrayOffset(int rowId) { + public int getArrayOffset(int rowId) { return arrayOffsets[rowId]; } @Override - public final void putArray(int rowId, int offset, int length) { + public void putArray(int rowId, int offset, int length) { arrayOffsets[rowId] = offset; arrayLengths[rowId] = length; } @Override - public final void loadBytes(ColumnVector.Array array) { + public void loadBytes(ColumnVector.Array array) { array.byteArray = byteData; array.byteArrayOffset = array.offset; } @@ -363,7 +363,7 @@ public final class OnHeapColumnVector extends ColumnVector { // @Override - public final int putByteArray(int rowId, byte[] value, int offset, int length) { + public int putByteArray(int rowId, byte[] value, int offset, int length) { int result = arrayData().appendBytes(length, value, offset); arrayOffsets[rowId] = result; arrayLengths[rowId] = length; @@ -371,12 +371,12 @@ public final class OnHeapColumnVector extends ColumnVector { } @Override - public final void reserve(int requiredCapacity) { + public void reserve(int requiredCapacity) { if (requiredCapacity > capacity) reserveInternal(requiredCapacity * 2); } // Spilt this function out since it is the slow path. - private final void reserveInternal(int newCapacity) { + private void reserveInternal(int newCapacity) { if (this.resultArray != null || DecimalType.isByteArrayDecimalType(type)) { int[] newLengths = new int[newCapacity]; int[] newOffsets = new int[newCapacity]; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java index ae9c8cc1ba..189cc3972c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java @@ -145,12 +145,13 @@ public class JavaApplySchemaSuite implements Serializable { Dataset<Row> df = sqlContext.createDataFrame(rowRDD, schema); df.registerTempTable("people"); - List<String> actual = sqlContext.sql("SELECT * FROM people").toJavaRDD().map(new Function<Row, String>() { - @Override - public String call(Row row) { - return row.getString(0) + "_" + row.get(1); - } - }).collect(); + List<String> actual = sqlContext.sql("SELECT * FROM people").toJavaRDD() + .map(new Function<Row, String>() { + @Override + public String call(Row row) { + return row.getString(0) + "_" + row.get(1); + } + }).collect(); List<String> expected = new ArrayList<>(2); expected.add("Michael_29"); 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 f3c5a86e20..cf764c645f 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 @@ -220,7 +220,8 @@ public class JavaDataFrameSuite { StructType schema1 = StructType$.MODULE$.apply(fields1); Assert.assertEquals(0, schema1.fieldIndex("id")); - List<StructField> fields2 = Arrays.asList(new StructField("id", DataTypes.StringType, true, Metadata.empty())); + List<StructField> fields2 = + Arrays.asList(new StructField("id", DataTypes.StringType, true, Metadata.empty())); StructType schema2 = StructType$.MODULE$.apply(fields2); Assert.assertEquals(0, schema2.fieldIndex("id")); } diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java index fc24600a1e..a8cbd4fab1 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java @@ -39,7 +39,7 @@ import java.util.BitSet; * does not contain union fields that are not supported by Spark SQL. */ -@SuppressWarnings({"ALL", "unchecked"}) +@SuppressWarnings("all") public class Complex implements org.apache.thrift.TBase<Complex, Complex._Fields>, java.io.Serializable, Cloneable { private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Complex"); diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 66448fd400..01f0c4de9e 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -349,7 +349,9 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaDStream<Integer> reducedWindowed; if (withInverse) { reducedWindowed = stream.reduceByWindow(new IntegerSum(), - new IntegerDifference(), new Duration(2000), new Duration(1000)); + new IntegerDifference(), + new Duration(2000), + new Duration(1000)); } else { reducedWindowed = stream.reduceByWindow(new IntegerSum(), new Duration(2000), new Duration(1000)); @@ -497,7 +499,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa pairStream.transformToPair( new Function2<JavaPairRDD<String, Integer>, Time, JavaPairRDD<String, String>>() { - @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in, Time time) { + @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in, + Time time) { return null; } } @@ -606,7 +609,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa pairStream1, new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time, JavaRDD<Double>>() { @Override - public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaPairRDD<String, Integer> rdd2, Time time) { + public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaPairRDD<String, Integer> rdd2, + Time time) { return null; } } @@ -616,7 +620,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa stream2, new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaPairRDD<Double, Double>>() { @Override - public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2, Time time) { + public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2, + Time time) { return null; } } @@ -624,9 +629,12 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa stream1.transformWithToPair( pairStream1, - new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time, JavaPairRDD<Double, Double>>() { + new Function3<JavaRDD<Integer>, JavaPairRDD<String, Integer>, Time, + JavaPairRDD<Double, Double>>() { @Override - public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, JavaPairRDD<String, Integer> rdd2, Time time) { + public JavaPairRDD<Double, Double> call(JavaRDD<Integer> rdd1, + JavaPairRDD<String, Integer> rdd2, + Time time) { return null; } } @@ -636,7 +644,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa stream2, new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() { @Override - public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, JavaRDD<String> rdd2, Time time) { + public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, JavaRDD<String> rdd2, + Time time) { return null; } } @@ -644,9 +653,12 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa pairStream1.transformWith( pairStream1, - new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<String, Integer>, Time, JavaRDD<Double>>() { + new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<String, Integer>, Time, + JavaRDD<Double>>() { @Override - public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, JavaPairRDD<String, Integer> rdd2, Time time) { + public JavaRDD<Double> call(JavaPairRDD<String, Integer> rdd1, + JavaPairRDD<String, Integer> rdd2, + Time time) { return null; } } @@ -654,9 +666,12 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa pairStream1.transformWithToPair( stream2, - new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time, JavaPairRDD<Double, Double>>() { + new Function3<JavaPairRDD<String, Integer>, JavaRDD<String>, Time, + JavaPairRDD<Double, Double>>() { @Override - public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1, JavaRDD<String> rdd2, Time time) { + public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1, + JavaRDD<String> rdd2, + Time time) { return null; } } @@ -664,9 +679,12 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa pairStream1.transformWithToPair( pairStream2, - new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<Double, Character>, Time, JavaPairRDD<Double, Double>>() { + new Function3<JavaPairRDD<String, Integer>, JavaPairRDD<Double, Character>, Time, + JavaPairRDD<Double, Double>>() { @Override - public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1, JavaPairRDD<Double, Character> rdd2, Time time) { + public JavaPairRDD<Double, Double> call(JavaPairRDD<String, Integer> rdd1, + JavaPairRDD<Double, Character> rdd2, + Time time) { return null; } } @@ -722,13 +740,16 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa listOfDStreams2, new Function2<List<JavaRDD<?>>, Time, JavaPairRDD<Integer, Tuple2<Integer, String>>>() { @Override - public JavaPairRDD<Integer, Tuple2<Integer, String>> call(List<JavaRDD<?>> listOfRDDs, Time time) { + public JavaPairRDD<Integer, Tuple2<Integer, String>> call(List<JavaRDD<?>> listOfRDDs, + Time time) { Assert.assertEquals(3, listOfRDDs.size()); JavaRDD<Integer> rdd1 = (JavaRDD<Integer>)listOfRDDs.get(0); JavaRDD<Integer> rdd2 = (JavaRDD<Integer>)listOfRDDs.get(1); - JavaRDD<Tuple2<Integer, String>> rdd3 = (JavaRDD<Tuple2<Integer, String>>)listOfRDDs.get(2); + JavaRDD<Tuple2<Integer, String>> rdd3 = + (JavaRDD<Tuple2<Integer, String>>)listOfRDDs.get(2); JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3); - PairFunction<Integer, Integer, Integer> mapToTuple = new PairFunction<Integer, Integer, Integer>() { + PairFunction<Integer, Integer, Integer> mapToTuple = + new PairFunction<Integer, Integer, Integer>() { @Override public Tuple2<Integer, Integer> call(Integer i) { return new Tuple2<>(i, i); @@ -739,7 +760,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa } ); JavaTestUtils.attachTestOutputStream(transformed2); - List<List<Tuple2<Integer, Tuple2<Integer, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List<List<Tuple2<Integer, Tuple2<Integer, String>>>> result = + JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); } @@ -981,7 +1003,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa new Tuple2<>(3, "new york"), new Tuple2<>(1, "new york"))); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<Integer, String> reversed = pairStream.mapToPair( new PairFunction<Tuple2<String, Integer>, Integer, String>() { @@ -1014,7 +1037,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa new Tuple2<>(3, "new york"), new Tuple2<>(1, "new york"))); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<Integer, String> reversed = pairStream.mapPartitionsToPair( new PairFlatMapFunction<Iterator<Tuple2<String, Integer>>, Integer, String>() { @@ -1044,7 +1068,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(1, 3, 4, 1), Arrays.asList(5, 5, 3, 1)); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaDStream<Integer> reversed = pairStream.map( new Function<Tuple2<String, Integer>, Integer>() { @@ -1116,7 +1141,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa new Tuple2<>("california", Arrays.asList("sharks", "ducks")), new Tuple2<>("new york", Arrays.asList("rangers", "islanders")))); - JavaDStream<Tuple2<String, String>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, String>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<String, Iterable<String>> grouped = pairStream.groupByKey(); @@ -1241,7 +1267,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ) ); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<String, Iterable<Integer>> groupWindowed = @@ -1255,7 +1282,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa } } - private static Set<Tuple2<String, HashSet<Integer>>> convert(List<Tuple2<String, List<Integer>>> listOfTuples) { + private static Set<Tuple2<String, HashSet<Integer>>> + convert(List<Tuple2<String, List<Integer>>> listOfTuples) { List<Tuple2<String, HashSet<Integer>>> newListOfTuples = new ArrayList<>(); for (Tuple2<String, List<Integer>> tuple: listOfTuples) { newListOfTuples.add(convert(tuple)); @@ -1280,7 +1308,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(new Tuple2<>("california", 10), new Tuple2<>("new york", 4))); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<String, Integer> reduceWindowed = @@ -1304,7 +1333,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(new Tuple2<>("california", 14), new Tuple2<>("new york", 9))); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey( @@ -1347,7 +1377,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(new Tuple2<>("california", 15), new Tuple2<>("new york", 11))); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey( @@ -1383,7 +1414,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(new Tuple2<>("california", 10), new Tuple2<>("new york", 4))); - JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream<Tuple2<String, Integer>> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<String, Integer> reduceWindowed = @@ -1630,19 +1662,27 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ssc, stringStringKVStream2, 1); JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream<String, Tuple2<Iterable<String>, Iterable<String>>> grouped = pairStream1.cogroup(pairStream2); + JavaPairDStream<String, Tuple2<Iterable<String>, Iterable<String>>> grouped = + pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); - List<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> result = + JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected.size(), result.size()); - Iterator<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> resultItr = result.iterator(); - Iterator<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expectedItr = expected.iterator(); + Iterator<List<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>>> resultItr = + result.iterator(); + Iterator<List<Tuple2<String, Tuple2<List<String>, List<String>>>>> expectedItr = + expected.iterator(); while (resultItr.hasNext() && expectedItr.hasNext()) { - Iterator<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>> resultElements = resultItr.next().iterator(); - Iterator<Tuple2<String, Tuple2<List<String>, List<String>>>> expectedElements = expectedItr.next().iterator(); + Iterator<Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>>> resultElements = + resultItr.next().iterator(); + Iterator<Tuple2<String, Tuple2<List<String>, List<String>>>> expectedElements = + expectedItr.next().iterator(); while (resultElements.hasNext() && expectedElements.hasNext()) { - Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>> resultElement = resultElements.next(); - Tuple2<String, Tuple2<List<String>, List<String>>> expectedElement = expectedElements.next(); + Tuple2<String, Tuple2<Iterable<String>, Iterable<String>>> resultElement = + resultElements.next(); + Tuple2<String, Tuple2<List<String>, List<String>>> expectedElement = + expectedElements.next(); Assert.assertEquals(expectedElement._1(), resultElement._1()); equalIterable(expectedElement._2()._1(), resultElement._2()._1()); equalIterable(expectedElement._2()._2(), resultElement._2()._2()); @@ -1719,7 +1759,8 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ssc, stringStringKVStream2, 1); JavaPairDStream<String, String> pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream<String, Tuple2<String, Optional<String>>> joined = pairStream1.leftOuterJoin(pairStream2); + JavaPairDStream<String, Tuple2<String, Optional<String>>> joined = + pairStream1.leftOuterJoin(pairStream2); JavaDStream<Long> counted = joined.count(); JavaTestUtils.attachTestOutputStream(counted); List<List<Long>> result = JavaTestUtils.runStreams(ssc, 2, 2); diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaStreamingListenerAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaStreamingListenerAPISuite.java index 67b2a0703e..ff0be820e0 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaStreamingListenerAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaStreamingListenerAPISuite.java @@ -77,12 +77,14 @@ public class JavaStreamingListenerAPISuite extends JavaStreamingListener { } @Override - public void onOutputOperationStarted(JavaStreamingListenerOutputOperationStarted outputOperationStarted) { + public void onOutputOperationStarted( + JavaStreamingListenerOutputOperationStarted outputOperationStarted) { super.onOutputOperationStarted(outputOperationStarted); } @Override - public void onOutputOperationCompleted(JavaStreamingListenerOutputOperationCompleted outputOperationCompleted) { + public void onOutputOperationCompleted( + JavaStreamingListenerOutputOperationCompleted outputOperationCompleted) { super.onOutputOperationCompleted(outputOperationCompleted); } } -- GitLab