From 1487c9af20a333ead55955acf4c0aa323bea0d07 Mon Sep 17 00:00:00 2001 From: Sean Owen <sowen@cloudera.com> Date: Sun, 19 Feb 2017 09:42:50 -0800 Subject: [PATCH] [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features ## What changes were proposed in this pull request? Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code. ## How was this patch tested? Jenkins tests Author: Sean Owen <sowen@cloudera.com> Closes #16964 from srowen/SPARK-19534. --- .../spark/network/TransportContext.java | 6 +- .../spark/network/util/MapConfigProvider.java | 8 +- .../network/ChunkFetchIntegrationSuite.java | 37 +- .../RequestTimeoutIntegrationSuite.java | 3 +- .../network/TransportClientFactorySuite.java | 51 +- .../TransportResponseHandlerSuite.java | 14 +- .../network/crypto/AuthIntegrationSuite.java | 19 +- .../spark/network/sasl/SparkSaslSuite.java | 65 +-- .../util/TransportFrameDecoderSuite.java | 44 +- .../network/sasl/SaslIntegrationSuite.java | 34 +- .../ExternalShuffleBlockHandlerSuite.java | 2 +- .../shuffle/ExternalShuffleCleanupSuite.java | 6 +- .../ExternalShuffleIntegrationSuite.java | 13 +- .../shuffle/OneForOneBlockFetcherSuite.java | 78 ++- .../shuffle/RetryingBlockFetcherSuite.java | 64 +-- .../unsafe/sort/UnsafeExternalSorter.java | 1 - .../org/apache/spark/JavaJdbcRDDSuite.java | 26 +- .../sort/UnsafeShuffleWriterSuite.java | 65 +-- .../map/AbstractBytesToBytesMapSuite.java | 25 +- .../sort/UnsafeExternalSorterSuite.java | 25 +- .../org/apache/spark/Java8RDDAPISuite.java | 7 +- .../test/org/apache/spark/JavaAPISuite.java | 492 ++++------------ .../kafka010/JavaConsumerStrategySuite.java | 24 +- .../SparkSubmitCommandBuilderSuite.java | 2 +- .../SparkSubmitOptionParserSuite.java | 8 +- .../apache/spark/ml/feature/JavaPCASuite.java | 35 +- .../classification/JavaNaiveBayesSuite.java | 10 +- .../clustering/JavaBisectingKMeansSuite.java | 4 +- .../spark/mllib/clustering/JavaLDASuite.java | 40 +- .../mllib/fpm/JavaAssociationRulesSuite.java | 6 +- .../regression/JavaLinearRegressionSuite.java | 11 +- .../mllib/tree/JavaDecisionTreeSuite.java | 15 +- .../SpecificParquetRecordReaderBase.java | 2 +- .../sql/Java8DatasetAggregatorSuite.java | 16 +- .../spark/sql/JavaApplySchemaSuite.java | 22 +- .../apache/spark/sql/JavaDataFrameSuite.java | 47 +- .../spark/sql/JavaDatasetAggregatorSuite.java | 49 +- .../sql/JavaDatasetAggregatorSuiteBase.java | 14 +- .../apache/spark/sql/JavaDatasetSuite.java | 147 ++--- .../org/apache/spark/sql/JavaUDFSuite.java | 37 +- .../streaming/JavaMapWithStateSuite.java | 81 +-- .../spark/streaming/JavaReceiverAPISuite.java | 24 +- .../streaming/JavaWriteAheadLogSuite.java | 10 +- .../apache/spark/streaming/Java8APISuite.java | 21 +- .../apache/spark/streaming/JavaAPISuite.java | 526 ++++-------------- 45 files changed, 662 insertions(+), 1574 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 37ba543380..965c4ae307 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 @@ -17,9 +17,9 @@ package org.apache.spark.network; +import java.util.ArrayList; import java.util.List; -import com.google.common.collect.Lists; import io.netty.channel.Channel; import io.netty.channel.socket.SocketChannel; import io.netty.handler.timeout.IdleStateHandler; @@ -100,7 +100,7 @@ public class TransportContext { } public TransportClientFactory createClientFactory() { - return createClientFactory(Lists.<TransportClientBootstrap>newArrayList()); + return createClientFactory(new ArrayList<>()); } /** Create a server which will attempt to bind to a specific port. */ @@ -120,7 +120,7 @@ public class TransportContext { } public TransportServer createServer() { - return createServer(0, Lists.<TransportServerBootstrap>newArrayList()); + return createServer(0, new ArrayList<>()); } public TransportChannelHandler initializePipeline(SocketChannel channel) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java b/common/network-common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java index b6667998b5..9cfee7f08d 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/MapConfigProvider.java @@ -17,22 +17,20 @@ package org.apache.spark.network.util; -import com.google.common.collect.Maps; - import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.NoSuchElementException; /** ConfigProvider based on a Map (copied in the constructor). */ public class MapConfigProvider extends ConfigProvider { - public static final MapConfigProvider EMPTY = new MapConfigProvider( - Collections.<String, String>emptyMap()); + public static final MapConfigProvider EMPTY = new MapConfigProvider(Collections.emptyMap()); private final Map<String, String> config; public MapConfigProvider(Map<String, String> config) { - this.config = Maps.newHashMap(config); + this.config = new HashMap<>(config); } @Override diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index 5bb8819132..824482af08 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -20,6 +20,7 @@ package org.apache.spark.network; import java.io.File; import java.io.RandomAccessFile; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.LinkedList; @@ -29,7 +30,6 @@ import java.util.Set; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.io.Closeables; import org.junit.AfterClass; @@ -179,49 +179,49 @@ public class ChunkFetchIntegrationSuite { @Test public void fetchBufferChunk() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX)); - assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks); assertTrue(res.failedChunks.isEmpty()); - assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers); res.releaseBuffers(); } @Test public void fetchFileChunk() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(FILE_CHUNK_INDEX)); - assertEquals(res.successChunks, Sets.newHashSet(FILE_CHUNK_INDEX)); + FetchResult res = fetchChunks(Arrays.asList(FILE_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(FILE_CHUNK_INDEX), res.successChunks); assertTrue(res.failedChunks.isEmpty()); - assertBufferListsEqual(res.buffers, Lists.newArrayList(fileChunk)); + assertBufferListsEqual(Arrays.asList(fileChunk), res.buffers); res.releaseBuffers(); } @Test public void fetchNonExistentChunk() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(12345)); + FetchResult res = fetchChunks(Arrays.asList(12345)); assertTrue(res.successChunks.isEmpty()); - assertEquals(res.failedChunks, Sets.newHashSet(12345)); + assertEquals(Sets.newHashSet(12345), res.failedChunks); assertTrue(res.buffers.isEmpty()); } @Test public void fetchBothChunks() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); - assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX), res.successChunks); assertTrue(res.failedChunks.isEmpty()); - assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk, fileChunk)); + assertBufferListsEqual(Arrays.asList(bufferChunk, fileChunk), res.buffers); res.releaseBuffers(); } @Test public void fetchChunkAndNonExistent() throws Exception { - FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, 12345)); - assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); - assertEquals(res.failedChunks, Sets.newHashSet(12345)); - assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, 12345)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks); + assertEquals(Sets.newHashSet(12345), res.failedChunks); + assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers); res.releaseBuffers(); } - private void assertBufferListsEqual(List<ManagedBuffer> list0, List<ManagedBuffer> list1) + private static void assertBufferListsEqual(List<ManagedBuffer> list0, List<ManagedBuffer> list1) throws Exception { assertEquals(list0.size(), list1.size()); for (int i = 0; i < list0.size(); i ++) { @@ -229,7 +229,8 @@ public class ChunkFetchIntegrationSuite { } } - private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception { + private static void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) + throws Exception { ByteBuffer nio0 = buffer0.nioByteBuffer(); ByteBuffer nio1 = buffer1.nioByteBuffer(); diff --git a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java index 959396bb8c..9aa17e24b6 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.network; -import com.google.common.collect.Maps; import com.google.common.util.concurrent.Uninterruptibles; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NioManagedBuffer; @@ -60,7 +59,7 @@ public class RequestTimeoutIntegrationSuite { @Before public void setUp() throws Exception { - Map<String, String> configMap = Maps.newHashMap(); + Map<String, String> configMap = new HashMap<>(); configMap.put("spark.shuffle.io.connectionTimeout", "10s"); conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 205ab88c84..e95d25fe6a 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -19,19 +19,20 @@ package org.apache.spark.network; import java.io.IOException; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import com.google.common.collect.Maps; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertTrue; import org.apache.spark.network.client.TransportClient; @@ -71,39 +72,36 @@ public class TransportClientFactorySuite { * * If concurrent is true, create multiple threads to create clients in parallel. */ - private void testClientReuse(final int maxConnections, boolean concurrent) + private void testClientReuse(int maxConnections, boolean concurrent) throws IOException, InterruptedException { - Map<String, String> configMap = Maps.newHashMap(); + Map<String, String> configMap = new HashMap<>(); configMap.put("spark.shuffle.io.numConnectionsPerPeer", Integer.toString(maxConnections)); TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); RpcHandler rpcHandler = new NoOpRpcHandler(); TransportContext context = new TransportContext(conf, rpcHandler); - final TransportClientFactory factory = context.createClientFactory(); - final Set<TransportClient> clients = Collections.synchronizedSet( + TransportClientFactory factory = context.createClientFactory(); + Set<TransportClient> clients = Collections.synchronizedSet( new HashSet<TransportClient>()); - final AtomicInteger failed = new AtomicInteger(); + AtomicInteger failed = new AtomicInteger(); Thread[] attempts = new Thread[maxConnections * 10]; // Launch a bunch of threads to create new clients. for (int i = 0; i < attempts.length; i++) { - attempts[i] = new Thread() { - @Override - public void run() { - try { - TransportClient client = - factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - assertTrue(client.isActive()); - clients.add(client); - } catch (IOException e) { - failed.incrementAndGet(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + attempts[i] = new Thread(() -> { + try { + TransportClient client = + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertTrue(client.isActive()); + clients.add(client); + } catch (IOException e) { + failed.incrementAndGet(); + } catch (InterruptedException e) { + throw new RuntimeException(e); } - }; + }); if (concurrent) { attempts[i].start(); @@ -113,8 +111,8 @@ public class TransportClientFactorySuite { } // Wait until all the threads complete. - for (int i = 0; i < attempts.length; i++) { - attempts[i].join(); + for (Thread attempt : attempts) { + attempt.join(); } Assert.assertEquals(0, failed.get()); @@ -150,7 +148,7 @@ public class TransportClientFactorySuite { TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); assertTrue(c1.isActive()); assertTrue(c2.isActive()); - assertTrue(c1 != c2); + assertNotSame(c1, c2); factory.close(); } @@ -167,7 +165,7 @@ public class TransportClientFactorySuite { assertFalse(c1.isActive()); TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - assertFalse(c1 == c2); + assertNotSame(c1, c2); assertTrue(c2.isActive()); factory.close(); } @@ -207,8 +205,7 @@ public class TransportClientFactorySuite { } }); TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); - TransportClientFactory factory = context.createClientFactory(); - try { + try (TransportClientFactory factory = context.createClientFactory()) { TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); assertTrue(c1.isActive()); long expiredTime = System.currentTimeMillis() + 10000; // 10 seconds @@ -216,8 +213,6 @@ public class TransportClientFactorySuite { Thread.sleep(10); } assertFalse(c1.isActive()); - } finally { - factory.close(); } } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java index 128f7cba74..4477c9a935 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java @@ -24,8 +24,6 @@ import io.netty.channel.local.LocalChannel; import org.junit.Test; import static org.junit.Assert.assertEquals; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; import static org.mockito.Mockito.*; import org.apache.spark.network.buffer.ManagedBuffer; @@ -54,7 +52,7 @@ public class TransportResponseHandlerSuite { assertEquals(1, handler.numOutstandingRequests()); handler.handle(new ChunkFetchSuccess(streamChunkId, new TestManagedBuffer(123))); - verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); + verify(callback, times(1)).onSuccess(eq(0), any()); assertEquals(0, handler.numOutstandingRequests()); } @@ -67,7 +65,7 @@ public class TransportResponseHandlerSuite { assertEquals(1, handler.numOutstandingRequests()); handler.handle(new ChunkFetchFailure(streamChunkId, "some error msg")); - verify(callback, times(1)).onFailure(eq(0), (Throwable) any()); + verify(callback, times(1)).onFailure(eq(0), any()); assertEquals(0, handler.numOutstandingRequests()); } @@ -84,9 +82,9 @@ public class TransportResponseHandlerSuite { handler.exceptionCaught(new Exception("duh duh duhhhh")); // should fail both b2 and b3 - verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); - verify(callback, times(1)).onFailure(eq(1), (Throwable) any()); - verify(callback, times(1)).onFailure(eq(2), (Throwable) any()); + verify(callback, times(1)).onSuccess(eq(0), any()); + verify(callback, times(1)).onFailure(eq(1), any()); + verify(callback, times(1)).onFailure(eq(2), any()); assertEquals(0, handler.numOutstandingRequests()); } @@ -118,7 +116,7 @@ public class TransportResponseHandlerSuite { assertEquals(1, handler.numOutstandingRequests()); handler.handle(new RpcFailure(12345, "oh no")); - verify(callback, times(1)).onFailure((Throwable) any()); + verify(callback, times(1)).onFailure(any()); assertEquals(0, handler.numOutstandingRequests()); } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java index 21609d5aa2..8751944a1c 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java @@ -18,11 +18,11 @@ package org.apache.spark.network.crypto; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.List; import java.util.Map; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import io.netty.channel.Channel; import org.junit.After; import org.junit.Test; @@ -163,20 +163,17 @@ public class AuthIntegrationSuite { } void createServer(String secret, boolean enableAes) throws Exception { - TransportServerBootstrap introspector = new TransportServerBootstrap() { - @Override - public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { - AuthTestCtx.this.serverChannel = channel; - if (rpcHandler instanceof AuthRpcHandler) { - AuthTestCtx.this.authRpcHandler = (AuthRpcHandler) rpcHandler; - } - return rpcHandler; + TransportServerBootstrap introspector = (channel, rpcHandler) -> { + this.serverChannel = channel; + if (rpcHandler instanceof AuthRpcHandler) { + this.authRpcHandler = (AuthRpcHandler) rpcHandler; } + return rpcHandler; }; SecretKeyHolder keyHolder = createKeyHolder(secret); TransportServerBootstrap auth = enableAes ? new AuthServerBootstrap(conf, keyHolder) : new SaslServerBootstrap(conf, keyHolder); - this.server = ctx.createServer(Lists.newArrayList(auth, introspector)); + this.server = ctx.createServer(Arrays.asList(auth, introspector)); } void createClient(String secret) throws Exception { @@ -186,7 +183,7 @@ public class AuthIntegrationSuite { void createClient(String secret, boolean enableAes) throws Exception { TransportConf clientConf = enableAes ? conf : new TransportConf("rpc", MapConfigProvider.EMPTY); - List<TransportClientBootstrap> bootstraps = Lists.<TransportClientBootstrap>newArrayList( + List<TransportClientBootstrap> bootstraps = Arrays.asList( new AuthClientBootstrap(clientConf, appId, createKeyHolder(secret))); this.client = ctx.createClientFactory(bootstraps) .createClient(TestUtils.getLocalHost(), server.getPort()); diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index 87129b900b..6f15718bd8 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -23,6 +23,7 @@ import static org.mockito.Mockito.*; import java.io.File; import java.lang.reflect.Method; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -35,7 +36,6 @@ import java.util.concurrent.atomic.AtomicReference; import javax.security.sasl.SaslException; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.io.ByteStreams; import com.google.common.io.Files; import io.netty.buffer.ByteBuf; @@ -45,8 +45,6 @@ import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelOutboundHandlerAdapter; import io.netty.channel.ChannelPromise; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.apache.spark.network.TestUtils; import org.apache.spark.network.TransportContext; @@ -137,18 +135,15 @@ public class SparkSaslSuite { testBasicSasl(true); } - private void testBasicSasl(boolean encrypt) throws Throwable { + private static void testBasicSasl(boolean encrypt) throws Throwable { RpcHandler rpcHandler = mock(RpcHandler.class); - doAnswer(new Answer<Void>() { - @Override - public Void answer(InvocationOnMock invocation) { - ByteBuffer message = (ByteBuffer) invocation.getArguments()[1]; - RpcResponseCallback cb = (RpcResponseCallback) invocation.getArguments()[2]; - assertEquals("Ping", JavaUtils.bytesToString(message)); - cb.onSuccess(JavaUtils.stringToBytes("Pong")); - return null; - } - }) + doAnswer(invocation -> { + ByteBuffer message = (ByteBuffer) invocation.getArguments()[1]; + RpcResponseCallback cb = (RpcResponseCallback) invocation.getArguments()[2]; + assertEquals("Ping", JavaUtils.bytesToString(message)); + cb.onSuccess(JavaUtils.stringToBytes("Pong")); + return null; + }) .when(rpcHandler) .receive(any(TransportClient.class), any(ByteBuffer.class), any(RpcResponseCallback.class)); @@ -255,21 +250,17 @@ public class SparkSaslSuite { @Test public void testFileRegionEncryption() throws Exception { - final Map<String, String> testConf = ImmutableMap.of( + Map<String, String> testConf = ImmutableMap.of( "spark.network.sasl.maxEncryptedBlockSize", "1k"); - final AtomicReference<ManagedBuffer> response = new AtomicReference<>(); - final File file = File.createTempFile("sasltest", ".txt"); + AtomicReference<ManagedBuffer> response = new AtomicReference<>(); + File file = File.createTempFile("sasltest", ".txt"); SaslTestCtx ctx = null; try { - final TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(testConf)); + TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(testConf)); StreamManager sm = mock(StreamManager.class); - when(sm.getChunk(anyLong(), anyInt())).thenAnswer(new Answer<ManagedBuffer>() { - @Override - public ManagedBuffer answer(InvocationOnMock invocation) { - return new FileSegmentManagedBuffer(conf, file, 0, file.length()); - } - }); + when(sm.getChunk(anyLong(), anyInt())).thenAnswer(invocation -> + new FileSegmentManagedBuffer(conf, file, 0, file.length())); RpcHandler rpcHandler = mock(RpcHandler.class); when(rpcHandler.getStreamManager()).thenReturn(sm); @@ -280,18 +271,15 @@ public class SparkSaslSuite { ctx = new SaslTestCtx(rpcHandler, true, false, testConf); - final CountDownLatch lock = new CountDownLatch(1); + CountDownLatch lock = new CountDownLatch(1); ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); - doAnswer(new Answer<Void>() { - @Override - public Void answer(InvocationOnMock invocation) { - response.set((ManagedBuffer) invocation.getArguments()[1]); - response.get().retain(); - lock.countDown(); - return null; - } - }).when(callback).onSuccess(anyInt(), any(ManagedBuffer.class)); + doAnswer(invocation -> { + response.set((ManagedBuffer) invocation.getArguments()[1]); + response.get().retain(); + lock.countDown(); + return null; + }).when(callback).onSuccess(anyInt(), any(ManagedBuffer.class)); ctx.client.fetchChunk(0, 0, callback); lock.await(10, TimeUnit.SECONDS); @@ -388,7 +376,7 @@ public class SparkSaslSuite { boolean disableClientEncryption) throws Exception { - this(rpcHandler, encrypt, disableClientEncryption, Collections.<String, String>emptyMap()); + this(rpcHandler, encrypt, disableClientEncryption, Collections.emptyMap()); } SaslTestCtx( @@ -416,7 +404,7 @@ public class SparkSaslSuite { checker)); try { - List<TransportClientBootstrap> clientBootstraps = Lists.newArrayList(); + List<TransportClientBootstrap> clientBootstraps = new ArrayList<>(); clientBootstraps.add(new SaslClientBootstrap(conf, "user", keyHolder)); if (disableClientEncryption) { clientBootstraps.add(new EncryptionDisablerBootstrap()); @@ -467,11 +455,6 @@ public class SparkSaslSuite { ctx.write(msg, promise); } - @Override - public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { - super.handlerRemoved(ctx); - } - @Override public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { channel.pipeline().addFirst("encryptionChecker", this); diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java index d4de4a941d..b53e413037 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java @@ -28,8 +28,6 @@ import io.netty.buffer.Unpooled; import io.netty.channel.ChannelHandlerContext; import org.junit.AfterClass; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import static org.junit.Assert.*; import static org.mockito.Mockito.*; @@ -52,7 +50,7 @@ public class TransportFrameDecoderSuite { @Test public void testInterception() throws Exception { - final int interceptedReads = 3; + int interceptedReads = 3; TransportFrameDecoder decoder = new TransportFrameDecoder(); TransportFrameDecoder.Interceptor interceptor = spy(new MockInterceptor(interceptedReads)); ChannelHandlerContext ctx = mockChannelHandlerContext(); @@ -84,22 +82,19 @@ public class TransportFrameDecoderSuite { public void testRetainedFrames() throws Exception { TransportFrameDecoder decoder = new TransportFrameDecoder(); - final AtomicInteger count = new AtomicInteger(); - final List<ByteBuf> retained = new ArrayList<>(); + AtomicInteger count = new AtomicInteger(); + List<ByteBuf> retained = new ArrayList<>(); ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); - when(ctx.fireChannelRead(any())).thenAnswer(new Answer<Void>() { - @Override - public Void answer(InvocationOnMock in) { - // Retain a few frames but not others. - ByteBuf buf = (ByteBuf) in.getArguments()[0]; - if (count.incrementAndGet() % 2 == 0) { - retained.add(buf); - } else { - buf.release(); - } - return null; + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + // Retain a few frames but not others. + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + if (count.incrementAndGet() % 2 == 0) { + retained.add(buf); + } else { + buf.release(); } + return null; }); ByteBuf data = createAndFeedFrames(100, decoder, ctx); @@ -150,12 +145,6 @@ public class TransportFrameDecoderSuite { testInvalidFrame(8); } - @Test(expected = IllegalArgumentException.class) - public void testLargeFrame() throws Exception { - // Frame length includes the frame size field, so need to add a few more bytes. - testInvalidFrame(Integer.MAX_VALUE + 9); - } - /** * Creates a number of randomly sized frames and feed them to the given decoder, verifying * that the frames were read. @@ -210,13 +199,10 @@ public class TransportFrameDecoderSuite { private ChannelHandlerContext mockChannelHandlerContext() { ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); - when(ctx.fireChannelRead(any())).thenAnswer(new Answer<Void>() { - @Override - public Void answer(InvocationOnMock in) { - ByteBuf buf = (ByteBuf) in.getArguments()[0]; - buf.release(); - return null; - } + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + buf.release(); + return null; }); return ctx; } 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 52f50a3409..c0e170e5b9 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 @@ -19,11 +19,11 @@ package org.apache.spark.network.sasl; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; -import com.google.common.collect.Lists; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -38,7 +38,6 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.ChunkReceivedCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.client.TransportClientBootstrap; import org.apache.spark.network.client.TransportClientFactory; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; @@ -105,8 +104,7 @@ public class SaslIntegrationSuite { @Test public void testGoodClient() throws IOException, InterruptedException { clientFactory = context.createClientFactory( - Lists.<TransportClientBootstrap>newArrayList( - new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); String msg = "Hello, World!"; @@ -120,8 +118,7 @@ public class SaslIntegrationSuite { when(badKeyHolder.getSaslUser(anyString())).thenReturn("other-app"); when(badKeyHolder.getSecretKey(anyString())).thenReturn("wrong-password"); clientFactory = context.createClientFactory( - Lists.<TransportClientBootstrap>newArrayList( - new SaslClientBootstrap(conf, "unknown-app", badKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "unknown-app", badKeyHolder))); try { // Bootstrap should fail on startup. @@ -134,8 +131,7 @@ public class SaslIntegrationSuite { @Test public void testNoSaslClient() throws IOException, InterruptedException { - clientFactory = context.createClientFactory( - Lists.<TransportClientBootstrap>newArrayList()); + clientFactory = context.createClientFactory(new ArrayList<>()); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); try { @@ -159,15 +155,11 @@ public class SaslIntegrationSuite { RpcHandler handler = new TestRpcHandler(); TransportContext context = new TransportContext(conf, handler); clientFactory = context.createClientFactory( - Lists.<TransportClientBootstrap>newArrayList( - new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); - TransportServer server = context.createServer(); - try { + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + try (TransportServer server = context.createServer()) { clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); } catch (Exception e) { assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); - } finally { - server.close(); } } @@ -191,14 +183,13 @@ public class SaslIntegrationSuite { try { // Create a client, and make a request to fetch blocks from a different app. clientFactory = blockServerContext.createClientFactory( - Lists.<TransportClientBootstrap>newArrayList( - new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); client1 = clientFactory.createClient(TestUtils.getLocalHost(), blockServer.getPort()); - final AtomicReference<Throwable> exception = new AtomicReference<>(); + AtomicReference<Throwable> exception = new AtomicReference<>(); - final CountDownLatch blockFetchLatch = new CountDownLatch(1); + CountDownLatch blockFetchLatch = new CountDownLatch(1); BlockFetchingListener listener = new BlockFetchingListener() { @Override public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { @@ -235,12 +226,11 @@ public class SaslIntegrationSuite { // Create a second client, authenticated with a different app ID, and try to read from // the stream created for the previous app. clientFactory2 = blockServerContext.createClientFactory( - Lists.<TransportClientBootstrap>newArrayList( - new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); + Arrays.asList(new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); client2 = clientFactory2.createClient(TestUtils.getLocalHost(), blockServer.getPort()); - final CountDownLatch chunkReceivedLatch = new CountDownLatch(1); + CountDownLatch chunkReceivedLatch = new CountDownLatch(1); ChunkReceivedCallback callback = new ChunkReceivedCallback() { @Override public void onSuccess(int chunkIndex, ManagedBuffer buffer) { @@ -284,7 +274,7 @@ public class SaslIntegrationSuite { } } - private void checkSecurityException(Throwable t) { + private static void checkSecurityException(Throwable t) { assertNotNull("No exception was caught.", t); assertTrue("Expected SecurityException.", t.getMessage().contains(SecurityException.class.getName())); 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 c036bc2e8d..e47a72c9d1 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 @@ -93,7 +93,7 @@ public class ExternalShuffleBlockHandlerSuite { ArgumentCaptor<ByteBuffer> response = ArgumentCaptor.forClass(ByteBuffer.class); verify(callback, times(1)).onSuccess(response.capture()); - verify(callback, never()).onFailure((Throwable) any()); + verify(callback, never()).onFailure(any()); StreamHandle handle = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response.getValue()); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 7757500b41..47c087088a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -60,12 +60,10 @@ public class ExternalShuffleCleanupSuite { public void cleanupUsesExecutor() throws IOException { TestShuffleDataContext dataContext = createSomeData(); - final AtomicBoolean cleanupCalled = new AtomicBoolean(false); + AtomicBoolean cleanupCalled = new AtomicBoolean(false); // Executor which does nothing to ensure we're actually using it. - Executor noThreadExecutor = new Executor() { - @Override public void execute(Runnable runnable) { cleanupCalled.set(true); } - }; + Executor noThreadExecutor = runnable -> cleanupCalled.set(true); ExternalShuffleBlockResolver manager = new ExternalShuffleBlockResolver(conf, null, noThreadExecutor); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 88de6fb83c..b8ae04eefb 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -19,6 +19,7 @@ package org.apache.spark.network.shuffle; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.LinkedList; @@ -29,7 +30,6 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.junit.After; import org.junit.AfterClass; @@ -173,7 +173,7 @@ public class ExternalShuffleIntegrationSuite { FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" }); assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks); assertTrue(exec0Fetch.failedBlocks.isEmpty()); - assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks[0])); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks[0])); exec0Fetch.releaseBuffers(); } @@ -185,7 +185,7 @@ public class ExternalShuffleIntegrationSuite { assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"), exec0Fetch.successBlocks); assertTrue(exec0Fetch.failedBlocks.isEmpty()); - assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks)); + assertBufferListsEqual(exec0Fetch.buffers, Arrays.asList(exec0Blocks)); exec0Fetch.releaseBuffers(); } @@ -241,7 +241,7 @@ public class ExternalShuffleIntegrationSuite { assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); } - private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) + private static void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) throws IOException, InterruptedException { ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false); client.init(APP_ID); @@ -249,7 +249,7 @@ public class ExternalShuffleIntegrationSuite { executorId, executorInfo); } - private void assertBufferListsEqual(List<ManagedBuffer> list0, List<byte[]> list1) + private static void assertBufferListsEqual(List<ManagedBuffer> list0, List<byte[]> list1) throws Exception { assertEquals(list0.size(), list1.size()); for (int i = 0; i < list0.size(); i ++) { @@ -257,7 +257,8 @@ public class ExternalShuffleIntegrationSuite { } } - private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception { + private static void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) + throws Exception { ByteBuffer nio0 = buffer0.nioByteBuffer(); ByteBuffer nio1 = buffer1.nioByteBuffer(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 2590b9ce4c..3e51fea3cf 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -25,8 +25,6 @@ import java.util.concurrent.atomic.AtomicInteger; import com.google.common.collect.Maps; import io.netty.buffer.Unpooled; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -85,8 +83,8 @@ public class OneForOneBlockFetcherSuite { // Each failure will cause a failure to be invoked in all remaining block fetches. verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); - verify(listener, times(1)).onBlockFetchFailure(eq("b1"), (Throwable) any()); - verify(listener, times(2)).onBlockFetchFailure(eq("b2"), (Throwable) any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); + verify(listener, times(2)).onBlockFetchFailure(eq("b2"), any()); } @Test @@ -100,15 +98,15 @@ public class OneForOneBlockFetcherSuite { // We may call both success and failure for the same block. verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); - verify(listener, times(1)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); verify(listener, times(1)).onBlockFetchSuccess("b2", blocks.get("b2")); - verify(listener, times(1)).onBlockFetchFailure(eq("b2"), (Throwable) any()); + verify(listener, times(1)).onBlockFetchFailure(eq("b2"), any()); } @Test public void testEmptyBlockFetch() { try { - fetchBlocks(Maps.<String, ManagedBuffer>newLinkedHashMap()); + fetchBlocks(Maps.newLinkedHashMap()); fail(); } catch (IllegalArgumentException e) { assertEquals("Zero-sized blockIds array", e.getMessage()); @@ -123,52 +121,46 @@ public class OneForOneBlockFetcherSuite { * * If a block's buffer is "null", an exception will be thrown instead. */ - private BlockFetchingListener fetchBlocks(final LinkedHashMap<String, ManagedBuffer> blocks) { + private static BlockFetchingListener fetchBlocks(LinkedHashMap<String, ManagedBuffer> blocks) { TransportClient client = mock(TransportClient.class); BlockFetchingListener listener = mock(BlockFetchingListener.class); - final String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); OneForOneBlockFetcher fetcher = new OneForOneBlockFetcher(client, "app-id", "exec-id", blockIds, listener); - // Respond to the "OpenBlocks" message with an appropirate ShuffleStreamHandle with streamId 123 - doAnswer(new Answer<Void>() { - @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - BlockTransferMessage message = BlockTransferMessage.Decoder.fromByteBuffer( - (ByteBuffer) invocationOnMock.getArguments()[0]); - RpcResponseCallback callback = (RpcResponseCallback) invocationOnMock.getArguments()[1]; - callback.onSuccess(new StreamHandle(123, blocks.size()).toByteBuffer()); - assertEquals(new OpenBlocks("app-id", "exec-id", blockIds), message); - return null; - } + // Respond to the "OpenBlocks" message with an appropriate ShuffleStreamHandle with streamId 123 + doAnswer(invocationOnMock -> { + BlockTransferMessage message = BlockTransferMessage.Decoder.fromByteBuffer( + (ByteBuffer) invocationOnMock.getArguments()[0]); + RpcResponseCallback callback = (RpcResponseCallback) invocationOnMock.getArguments()[1]; + callback.onSuccess(new StreamHandle(123, blocks.size()).toByteBuffer()); + assertEquals(new OpenBlocks("app-id", "exec-id", blockIds), message); + return null; }).when(client).sendRpc(any(ByteBuffer.class), any(RpcResponseCallback.class)); // Respond to each chunk request with a single buffer from our blocks array. - final AtomicInteger expectedChunkIndex = new AtomicInteger(0); - final Iterator<ManagedBuffer> blockIterator = blocks.values().iterator(); - doAnswer(new Answer<Void>() { - @Override - public Void answer(InvocationOnMock invocation) throws Throwable { - try { - long streamId = (Long) invocation.getArguments()[0]; - int myChunkIndex = (Integer) invocation.getArguments()[1]; - assertEquals(123, streamId); - assertEquals(expectedChunkIndex.getAndIncrement(), myChunkIndex); - - ChunkReceivedCallback callback = (ChunkReceivedCallback) invocation.getArguments()[2]; - ManagedBuffer result = blockIterator.next(); - if (result != null) { - callback.onSuccess(myChunkIndex, result); - } else { - callback.onFailure(myChunkIndex, new RuntimeException("Failed " + myChunkIndex)); - } - } catch (Exception e) { - e.printStackTrace(); - fail("Unexpected failure"); + AtomicInteger expectedChunkIndex = new AtomicInteger(0); + Iterator<ManagedBuffer> blockIterator = blocks.values().iterator(); + doAnswer(invocation -> { + try { + long streamId = (Long) invocation.getArguments()[0]; + int myChunkIndex = (Integer) invocation.getArguments()[1]; + assertEquals(123, streamId); + assertEquals(expectedChunkIndex.getAndIncrement(), myChunkIndex); + + ChunkReceivedCallback callback = (ChunkReceivedCallback) invocation.getArguments()[2]; + ManagedBuffer result = blockIterator.next(); + if (result != null) { + callback.onSuccess(myChunkIndex, result); + } else { + callback.onFailure(myChunkIndex, new RuntimeException("Failed " + myChunkIndex)); } - return null; + } catch (Exception e) { + e.printStackTrace(); + fail("Unexpected failure"); } - }).when(client).fetchChunk(anyLong(), anyInt(), (ChunkReceivedCallback) any()); + return null; + }).when(client).fetchChunk(anyLong(), anyInt(), any()); fetcher.start(); return listener; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java index 6db71eea6e..a530e16734 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -28,7 +28,6 @@ import java.util.Map; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import org.junit.Test; -import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import org.mockito.stubbing.Stubber; @@ -84,7 +83,7 @@ public class RetryingBlockFetcherSuite { performInteractions(interactions, listener); - verify(listener).onBlockFetchFailure(eq("b0"), (Throwable) any()); + verify(listener).onBlockFetchFailure(eq("b0"), any()); verify(listener).onBlockFetchSuccess("b1", block1); verifyNoMoreInteractions(listener); } @@ -190,7 +189,7 @@ public class RetryingBlockFetcherSuite { performInteractions(interactions, listener); verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); - verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), any()); verifyNoMoreInteractions(listener); } @@ -220,7 +219,7 @@ public class RetryingBlockFetcherSuite { performInteractions(interactions, listener); verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); - verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), any()); verify(listener, timeout(5000)).onBlockFetchSuccess("b2", block2); verifyNoMoreInteractions(listener); } @@ -249,40 +248,37 @@ public class RetryingBlockFetcherSuite { Stubber stub = null; // Contains all blockIds that are referenced across all interactions. - final LinkedHashSet<String> blockIds = Sets.newLinkedHashSet(); + LinkedHashSet<String> blockIds = Sets.newLinkedHashSet(); - for (final Map<String, Object> interaction : interactions) { + for (Map<String, Object> interaction : interactions) { blockIds.addAll(interaction.keySet()); - Answer<Void> answer = new Answer<Void>() { - @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - try { - // Verify that the RetryingBlockFetcher requested the expected blocks. - String[] requestedBlockIds = (String[]) invocationOnMock.getArguments()[0]; - String[] desiredBlockIds = interaction.keySet().toArray(new String[interaction.size()]); - assertArrayEquals(desiredBlockIds, requestedBlockIds); - - // Now actually invoke the success/failure callbacks on each block. - BlockFetchingListener retryListener = - (BlockFetchingListener) invocationOnMock.getArguments()[1]; - for (Map.Entry<String, Object> block : interaction.entrySet()) { - String blockId = block.getKey(); - Object blockValue = block.getValue(); - - if (blockValue instanceof ManagedBuffer) { - retryListener.onBlockFetchSuccess(blockId, (ManagedBuffer) blockValue); - } else if (blockValue instanceof Exception) { - retryListener.onBlockFetchFailure(blockId, (Exception) blockValue); - } else { - fail("Can only handle ManagedBuffers and Exceptions, got " + blockValue); - } + Answer<Void> answer = invocationOnMock -> { + try { + // Verify that the RetryingBlockFetcher requested the expected blocks. + String[] requestedBlockIds = (String[]) invocationOnMock.getArguments()[0]; + String[] desiredBlockIds = interaction.keySet().toArray(new String[interaction.size()]); + assertArrayEquals(desiredBlockIds, requestedBlockIds); + + // Now actually invoke the success/failure callbacks on each block. + BlockFetchingListener retryListener = + (BlockFetchingListener) invocationOnMock.getArguments()[1]; + for (Map.Entry<String, Object> block : interaction.entrySet()) { + String blockId = block.getKey(); + Object blockValue = block.getValue(); + + if (blockValue instanceof ManagedBuffer) { + retryListener.onBlockFetchSuccess(blockId, (ManagedBuffer) blockValue); + } else if (blockValue instanceof Exception) { + retryListener.onBlockFetchFailure(blockId, (Exception) blockValue); + } else { + fail("Can only handle ManagedBuffers and Exceptions, got " + blockValue); } - return null; - } catch (Throwable e) { - e.printStackTrace(); - throw e; } + return null; + } catch (Throwable e) { + e.printStackTrace(); + throw e; } }; @@ -295,7 +291,7 @@ public class RetryingBlockFetcherSuite { } assertNotNull(stub); - stub.when(fetchStarter).createAndStart((String[]) any(), (BlockFetchingListener) anyObject()); + stub.when(fetchStarter).createAndStart(any(), anyObject()); String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]); new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start(); } 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 189d607fa6..29aca04a3d 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 @@ -37,7 +37,6 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.UnsafeAlignedOffset; import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.util.TaskCompletionListener; import org.apache.spark.util.Utils; /** diff --git a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java index 7fe452a48d..a6589d2898 100644 --- a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java +++ b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java @@ -20,14 +20,11 @@ import java.io.Serializable; import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; -import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.Function2; import org.apache.spark.rdd.JdbcRDD; import org.junit.After; import org.junit.Assert; @@ -89,30 +86,13 @@ public class JavaJdbcRDDSuite implements Serializable { public void testJavaJdbcRDD() throws Exception { JavaRDD<Integer> rdd = JdbcRDD.create( sc, - new JdbcRDD.ConnectionFactory() { - @Override - public Connection getConnection() throws SQLException { - return DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb"); - } - }, + () -> DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb"), "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?", 1, 100, 1, - new Function<ResultSet, Integer>() { - @Override - public Integer call(ResultSet r) throws Exception { - return r.getInt(1); - } - } + r -> r.getInt(1) ).cache(); Assert.assertEquals(100, rdd.count()); - Assert.assertEquals( - Integer.valueOf(10100), - rdd.reduce(new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - })); + Assert.assertEquals(Integer.valueOf(10100), rdd.reduce((i1, i2) -> i1 + i2)); } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 088b68132d..24a55df84a 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -34,8 +34,6 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.apache.spark.HashPartitioner; import org.apache.spark.ShuffleDependency; @@ -119,9 +117,7 @@ public class UnsafeShuffleWriterSuite { any(File.class), any(SerializerInstance.class), anyInt(), - any(ShuffleWriteMetrics.class))).thenAnswer(new Answer<DiskBlockObjectWriter>() { - @Override - public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { + any(ShuffleWriteMetrics.class))).thenAnswer(invocationOnMock -> { Object[] args = invocationOnMock.getArguments(); return new DiskBlockObjectWriter( (File) args[1], @@ -132,33 +128,24 @@ public class UnsafeShuffleWriterSuite { (ShuffleWriteMetrics) args[4], (BlockId) args[0] ); - } - }); + }); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); - doAnswer(new Answer<Void>() { - @Override - public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; - File tmp = (File) invocationOnMock.getArguments()[3]; - mergedOutputFile.delete(); - tmp.renameTo(mergedOutputFile); - return null; - } + doAnswer(invocationOnMock -> { + partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; + File tmp = (File) invocationOnMock.getArguments()[3]; + mergedOutputFile.delete(); + tmp.renameTo(mergedOutputFile); + return null; }).when(shuffleBlockResolver) .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); - when(diskBlockManager.createTempShuffleBlock()).thenAnswer( - new Answer<Tuple2<TempShuffleBlockId, File>>() { - @Override - public Tuple2<TempShuffleBlockId, File> answer( - InvocationOnMock invocationOnMock) throws Throwable { - TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); - File file = File.createTempFile("spillFile", ".spill", tempDir); - spillFilesCreated.add(file); - return Tuple2$.MODULE$.apply(blockId, file); - } - }); + when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { + TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); + File file = File.createTempFile("spillFile", ".spill", tempDir); + spillFilesCreated.add(file); + return Tuple2$.MODULE$.apply(blockId, file); + }); when(taskContext.taskMetrics()).thenReturn(taskMetrics); when(shuffleDep.serializer()).thenReturn(serializer); @@ -243,7 +230,7 @@ public class UnsafeShuffleWriterSuite { @Test public void writeEmptyIterator() throws Exception { final UnsafeShuffleWriter<Object, Object> writer = createWriter(true); - writer.write(Iterators.<Product2<Object, Object>>emptyIterator()); + writer.write(Iterators.emptyIterator()); final Option<MapStatus> mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); @@ -259,7 +246,7 @@ public class UnsafeShuffleWriterSuite { // In this example, each partition should have exactly one record: final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>(); for (int i = 0; i < NUM_PARTITITONS; i++) { - dataToWrite.add(new Tuple2<Object, Object>(i, i)); + dataToWrite.add(new Tuple2<>(i, i)); } final UnsafeShuffleWriter<Object, Object> writer = createWriter(true); writer.write(dataToWrite.iterator()); @@ -315,7 +302,7 @@ public class UnsafeShuffleWriterSuite { final UnsafeShuffleWriter<Object, Object> writer = createWriter(transferToEnabled); final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>(); for (int i : new int[] { 1, 2, 3, 4, 4, 2 }) { - dataToWrite.add(new Tuple2<Object, Object>(i, i)); + dataToWrite.add(new Tuple2<>(i, i)); } writer.insertRecordIntoSorter(dataToWrite.get(0)); writer.insertRecordIntoSorter(dataToWrite.get(1)); @@ -424,7 +411,7 @@ public class UnsafeShuffleWriterSuite { final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>(); final byte[] bigByteArray = new byte[PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES / 10]; for (int i = 0; i < 10 + 1; i++) { - dataToWrite.add(new Tuple2<Object, Object>(i, bigByteArray)); + dataToWrite.add(new Tuple2<>(i, bigByteArray)); } writer.write(dataToWrite.iterator()); assertEquals(2, spillFilesCreated.size()); @@ -458,7 +445,7 @@ public class UnsafeShuffleWriterSuite { final UnsafeShuffleWriter<Object, Object> writer = createWriter(false); final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>(); for (int i = 0; i < UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { - dataToWrite.add(new Tuple2<Object, Object>(i, i)); + dataToWrite.add(new Tuple2<>(i, i)); } writer.write(dataToWrite.iterator()); writer.stop(true); @@ -478,7 +465,7 @@ public class UnsafeShuffleWriterSuite { final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>(); final byte[] bytes = new byte[(int) (ShuffleExternalSorter.DISK_WRITE_BUFFER_SIZE * 2.5)]; new Random(42).nextBytes(bytes); - dataToWrite.add(new Tuple2<Object, Object>(1, ByteBuffer.wrap(bytes))); + dataToWrite.add(new Tuple2<>(1, ByteBuffer.wrap(bytes))); writer.write(dataToWrite.iterator()); writer.stop(true); assertEquals( @@ -491,15 +478,15 @@ public class UnsafeShuffleWriterSuite { public void writeRecordsThatAreBiggerThanMaxRecordSize() throws Exception { final UnsafeShuffleWriter<Object, Object> writer = createWriter(false); final ArrayList<Product2<Object, Object>> dataToWrite = new ArrayList<>(); - dataToWrite.add(new Tuple2<Object, Object>(1, ByteBuffer.wrap(new byte[1]))); + dataToWrite.add(new Tuple2<>(1, ByteBuffer.wrap(new byte[1]))); // We should be able to write a record that's right _at_ the max record size final byte[] atMaxRecordSize = new byte[(int) taskMemoryManager.pageSizeBytes() - 4]; new Random(42).nextBytes(atMaxRecordSize); - dataToWrite.add(new Tuple2<Object, Object>(2, ByteBuffer.wrap(atMaxRecordSize))); + dataToWrite.add(new Tuple2<>(2, ByteBuffer.wrap(atMaxRecordSize))); // Inserting a record that's larger than the max record size final byte[] exceedsMaxRecordSize = new byte[(int) taskMemoryManager.pageSizeBytes()]; new Random(42).nextBytes(exceedsMaxRecordSize); - dataToWrite.add(new Tuple2<Object, Object>(3, ByteBuffer.wrap(exceedsMaxRecordSize))); + dataToWrite.add(new Tuple2<>(3, ByteBuffer.wrap(exceedsMaxRecordSize))); writer.write(dataToWrite.iterator()); writer.stop(true); assertEquals( @@ -511,10 +498,10 @@ public class UnsafeShuffleWriterSuite { @Test public void spillFilesAreDeletedWhenStoppingAfterError() throws IOException { final UnsafeShuffleWriter<Object, Object> writer = createWriter(false); - writer.insertRecordIntoSorter(new Tuple2<Object, Object>(1, 1)); - writer.insertRecordIntoSorter(new Tuple2<Object, Object>(2, 2)); + writer.insertRecordIntoSorter(new Tuple2<>(1, 1)); + writer.insertRecordIntoSorter(new Tuple2<>(2, 2)); writer.forceSorterToSpill(); - writer.insertRecordIntoSorter(new Tuple2<Object, Object>(2, 2)); + writer.insertRecordIntoSorter(new Tuple2<>(2, 2)); writer.stop(false); assertSpillFilesWereCleanedUp(); } 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 26568146bf..03cec8ed81 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 @@ -22,7 +22,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; -import scala.Tuple2; import scala.Tuple2$; import org.junit.After; @@ -31,8 +30,6 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.apache.spark.SparkConf; import org.apache.spark.executor.ShuffleWriteMetrics; @@ -88,25 +85,18 @@ public abstract class AbstractBytesToBytesMapSuite { spillFilesCreated.clear(); MockitoAnnotations.initMocks(this); when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); - when(diskBlockManager.createTempLocalBlock()).thenAnswer( - new Answer<Tuple2<TempLocalBlockId, File>>() { - @Override - 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); - return Tuple2$.MODULE$.apply(blockId, file); - } + when(diskBlockManager.createTempLocalBlock()).thenAnswer(invocationOnMock -> { + TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); + File file = File.createTempFile("spillFile", ".spill", tempDir); + spillFilesCreated.add(file); + return Tuple2$.MODULE$.apply(blockId, file); }); when(blockManager.getDiskWriter( any(BlockId.class), any(File.class), any(SerializerInstance.class), anyInt(), - any(ShuffleWriteMetrics.class))).thenAnswer(new Answer<DiskBlockObjectWriter>() { - @Override - public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { + any(ShuffleWriteMetrics.class))).thenAnswer(invocationOnMock -> { Object[] args = invocationOnMock.getArguments(); return new DiskBlockObjectWriter( @@ -118,8 +108,7 @@ public abstract class AbstractBytesToBytesMapSuite { (ShuffleWriteMetrics) args[4], (BlockId) args[0] ); - } - }); + }); } @After 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 fbbe530a13..771d39016c 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 @@ -23,7 +23,6 @@ import java.util.Arrays; import java.util.LinkedList; import java.util.UUID; -import scala.Tuple2; import scala.Tuple2$; import org.junit.After; @@ -31,8 +30,6 @@ import org.junit.Before; import org.junit.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; -import org.mockito.stubbing.Answer; import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; @@ -96,25 +93,18 @@ 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>>() { - @Override - 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); - return Tuple2$.MODULE$.apply(blockId, file); - } + when(diskBlockManager.createTempLocalBlock()).thenAnswer(invocationOnMock -> { + TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); + File file = File.createTempFile("spillFile", ".spill", tempDir); + spillFilesCreated.add(file); + return Tuple2$.MODULE$.apply(blockId, file); }); when(blockManager.getDiskWriter( any(BlockId.class), any(File.class), any(SerializerInstance.class), anyInt(), - any(ShuffleWriteMetrics.class))).thenAnswer(new Answer<DiskBlockObjectWriter>() { - @Override - public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { + any(ShuffleWriteMetrics.class))).thenAnswer(invocationOnMock -> { Object[] args = invocationOnMock.getArguments(); return new DiskBlockObjectWriter( @@ -126,8 +116,7 @@ public class UnsafeExternalSorterSuite { (ShuffleWriteMetrics) args[4], (BlockId) args[0] ); - } - }); + }); } @After diff --git a/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java b/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java index e22ad89c1d..1d2b05ebc2 100644 --- a/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/Java8RDDAPISuite.java @@ -64,12 +64,7 @@ public class Java8RDDAPISuite implements Serializable { public void foreachWithAnonymousClass() { foreachCalls = 0; JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World")); - rdd.foreach(new VoidFunction<String>() { - @Override - public void call(String s) { - foreachCalls++; - } - }); + rdd.foreach(s -> foreachCalls++); Assert.assertEquals(2, foreachCalls); } diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index 80aab100ac..512149127d 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -31,7 +31,6 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.*; import org.apache.spark.Accumulator; @@ -208,7 +207,7 @@ public class JavaAPISuite implements Serializable { assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); // Custom comparator - sortedRDD = rdd.sortByKey(Collections.<Integer>reverseOrder(), false); + sortedRDD = rdd.sortByKey(Collections.reverseOrder(), false); assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); assertEquals(new Tuple2<>(0, 4), sortedPairs.get(1)); @@ -266,13 +265,7 @@ 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>() { - @Override - public Integer call(Tuple2<Integer, Integer> t) { - return t._1(); - } - }, true, 2); + JavaRDD<Tuple2<Integer, Integer>> sortedRDD = rdd.sortBy(Tuple2::_1, true, 2); assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); List<Tuple2<Integer, Integer>> sortedPairs = sortedRDD.collect(); @@ -280,12 +273,7 @@ public class JavaAPISuite implements Serializable { assertEquals(new Tuple2<>(3, 2), sortedPairs.get(2)); // compare on second value - sortedRDD = rdd.sortBy(new Function<Tuple2<Integer, Integer>, Integer>() { - @Override - public Integer call(Tuple2<Integer, Integer> t) { - return t._2(); - } - }, true, 2); + sortedRDD = rdd.sortBy(Tuple2::_2, true, 2); assertEquals(new Tuple2<>(-1, 1), sortedRDD.first()); sortedPairs = sortedRDD.collect(); assertEquals(new Tuple2<>(3, 2), sortedPairs.get(1)); @@ -294,28 +282,20 @@ public class JavaAPISuite implements Serializable { @Test public void foreach() { - final LongAccumulator accum = sc.sc().longAccumulator(); + LongAccumulator accum = sc.sc().longAccumulator(); JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World")); - rdd.foreach(new VoidFunction<String>() { - @Override - public void call(String s) { - accum.add(1); - } - }); + rdd.foreach(s -> accum.add(1)); assertEquals(2, accum.value().intValue()); } @Test public void foreachPartition() { - final LongAccumulator accum = sc.sc().longAccumulator(); + LongAccumulator accum = sc.sc().longAccumulator(); JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello", "World")); - rdd.foreachPartition(new VoidFunction<Iterator<String>>() { - @Override - public void call(Iterator<String> iter) { - while (iter.hasNext()) { - iter.next(); - accum.add(1); - } + rdd.foreachPartition(iter -> { + while (iter.hasNext()) { + iter.next(); + accum.add(1); } }); assertEquals(2, accum.value().intValue()); @@ -361,12 +341,7 @@ public class JavaAPISuite implements Serializable { @Test public void groupBy() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function<Integer, Boolean> isOdd = new Function<Integer, Boolean>() { - @Override - public Boolean call(Integer x) { - return x % 2 == 0; - } - }; + Function<Integer, Boolean> isOdd = x -> x % 2 == 0; JavaPairRDD<Boolean, Iterable<Integer>> oddsAndEvens = rdd.groupBy(isOdd); assertEquals(2, oddsAndEvens.count()); assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens @@ -383,12 +358,7 @@ public class JavaAPISuite implements Serializable { // Regression test for SPARK-4459 JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Function<Tuple2<Integer, Integer>, Boolean> areOdd = - new Function<Tuple2<Integer, Integer>, Boolean>() { - @Override - public Boolean call(Tuple2<Integer, Integer> x) { - return (x._1() % 2 == 0) && (x._2() % 2 == 0); - } - }; + x -> (x._1() % 2 == 0) && (x._2() % 2 == 0); JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd); JavaPairRDD<Boolean, Iterable<Tuple2<Integer, Integer>>> oddsAndEvens = pairRDD.groupBy(areOdd); assertEquals(2, oddsAndEvens.count()); @@ -406,13 +376,7 @@ public class JavaAPISuite implements Serializable { public void keyByOnPairRDD() { // Regression test for SPARK-4459 JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function<Tuple2<Integer, Integer>, String> sumToString = - new Function<Tuple2<Integer, Integer>, String>() { - @Override - public String call(Tuple2<Integer, Integer> x) { - return String.valueOf(x._1() + x._2()); - } - }; + Function<Tuple2<Integer, Integer>, String> sumToString = x -> String.valueOf(x._1() + x._2()); JavaPairRDD<Integer, Integer> pairRDD = rdd.zip(rdd); JavaPairRDD<String, Tuple2<Integer, Integer>> keyed = pairRDD.keyBy(sumToString); assertEquals(7, keyed.count()); @@ -516,25 +480,14 @@ public class JavaAPISuite implements Serializable { rdd1.leftOuterJoin(rdd2).collect(); assertEquals(5, joined.size()); Tuple2<Integer,Tuple2<Integer,Optional<Character>>> firstUnmatched = - rdd1.leftOuterJoin(rdd2).filter( - new Function<Tuple2<Integer, Tuple2<Integer, Optional<Character>>>, Boolean>() { - @Override - public Boolean call(Tuple2<Integer, Tuple2<Integer, Optional<Character>>> tup) { - return !tup._2()._2().isPresent(); - } - }).first(); + rdd1.leftOuterJoin(rdd2).filter(tup -> !tup._2()._2().isPresent()).first(); assertEquals(3, firstUnmatched._1().intValue()); } @Test public void foldReduce() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); - Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }; + Function2<Integer, Integer, Integer> add = (a, b) -> a + b; int sum = rdd.fold(0, add); assertEquals(33, sum); @@ -546,12 +499,7 @@ public class JavaAPISuite implements Serializable { @Test public void treeReduce() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10); - Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }; + Function2<Integer, Integer, Integer> add = (a, b) -> a + b; for (int depth = 1; depth <= 10; depth++) { int sum = rdd.treeReduce(add, depth); assertEquals(-5, sum); @@ -561,12 +509,7 @@ public class JavaAPISuite implements Serializable { @Test public void treeAggregate() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(-5, -4, -3, -2, -1, 1, 2, 3, 4), 10); - Function2<Integer, Integer, Integer> add = new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }; + Function2<Integer, Integer, Integer> add = (a, b) -> a + b; for (int depth = 1; depth <= 10; depth++) { int sum = rdd.treeAggregate(0, add, add, depth); assertEquals(-5, sum); @@ -584,21 +527,15 @@ public class JavaAPISuite implements Serializable { new Tuple2<>(5, 1), new Tuple2<>(5, 3)), 2); - Map<Integer, Set<Integer>> sets = pairs.aggregateByKey(new HashSet<Integer>(), - new Function2<Set<Integer>, Integer, Set<Integer>>() { - @Override - public Set<Integer> call(Set<Integer> a, Integer b) { - a.add(b); - return a; - } - }, - new Function2<Set<Integer>, Set<Integer>, Set<Integer>>() { - @Override - public Set<Integer> call(Set<Integer> a, Set<Integer> b) { - a.addAll(b); - return a; - } - }).collectAsMap(); + Map<Integer, HashSet<Integer>> sets = pairs.aggregateByKey(new HashSet<Integer>(), + (a, b) -> { + a.add(b); + return a; + }, + (a, b) -> { + a.addAll(b); + return a; + }).collectAsMap(); assertEquals(3, sets.size()); assertEquals(new HashSet<>(Arrays.asList(1)), sets.get(1)); assertEquals(new HashSet<>(Arrays.asList(2)), sets.get(3)); @@ -616,13 +553,7 @@ public class JavaAPISuite implements Serializable { new Tuple2<>(3, 1) ); JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs); - JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0, - new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }); + JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0, (a, b) -> a + b); assertEquals(1, sums.lookup(1).get(0).intValue()); assertEquals(2, sums.lookup(2).get(0).intValue()); assertEquals(3, sums.lookup(3).get(0).intValue()); @@ -639,13 +570,7 @@ public class JavaAPISuite implements Serializable { new Tuple2<>(3, 1) ); JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs); - JavaPairRDD<Integer, Integer> counts = rdd.reduceByKey( - new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }); + JavaPairRDD<Integer, Integer> counts = rdd.reduceByKey((a, b) -> a + b); assertEquals(1, counts.lookup(1).get(0).intValue()); assertEquals(2, counts.lookup(2).get(0).intValue()); assertEquals(3, counts.lookup(3).get(0).intValue()); @@ -655,12 +580,7 @@ public class JavaAPISuite implements Serializable { assertEquals(2, localCounts.get(2).intValue()); assertEquals(3, localCounts.get(3).intValue()); - localCounts = rdd.reduceByKeyLocally(new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }); + localCounts = rdd.reduceByKeyLocally((a, b) -> a + b); assertEquals(1, localCounts.get(1).intValue()); assertEquals(2, localCounts.get(2).intValue()); assertEquals(3, localCounts.get(3).intValue()); @@ -692,20 +612,8 @@ public class JavaAPISuite implements Serializable { 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()); - assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter( - new Function<Integer, Boolean>() { - @Override - public Boolean call(Integer i) { - return i > 1; - } - }).isEmpty()); + assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter(i -> i < 0).isEmpty()); + assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter(i -> i > 1).isEmpty()); } @Test @@ -721,12 +629,7 @@ public class JavaAPISuite implements Serializable { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); JavaDoubleRDD distinct = rdd.distinct(); assertEquals(5, distinct.count()); - JavaDoubleRDD filter = rdd.filter(new Function<Double, Boolean>() { - @Override - public Boolean call(Double x) { - return x > 2.0; - } - }); + JavaDoubleRDD filter = rdd.filter(x -> x > 2.0); assertEquals(3, filter.count()); JavaDoubleRDD union = rdd.union(rdd); assertEquals(12, union.count()); @@ -764,7 +667,7 @@ public class JavaAPISuite implements Serializable { // SPARK-5744 assertArrayEquals( new long[] {0}, - sc.parallelizeDoubles(new ArrayList<Double>(0), 1).histogram(new double[]{0.0, 1.0})); + sc.parallelizeDoubles(new ArrayList<>(0), 1).histogram(new double[]{0.0, 1.0})); } private static class DoubleComparator implements Comparator<Double>, Serializable { @@ -833,12 +736,7 @@ public class JavaAPISuite implements Serializable { @Test public void reduceOnJavaDoubleRDD() { JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - double sum = rdd.reduce(new Function2<Double, Double, Double>() { - @Override - public Double call(Double v1, Double v2) { - return v1 + v2; - } - }); + double sum = rdd.reduce((v1, v2) -> v1 + v2); assertEquals(10.0, sum, 0.001); } @@ -859,27 +757,11 @@ public class JavaAPISuite implements Serializable { @Test public void map() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction<Integer>() { - @Override - public double call(Integer x) { - return x.doubleValue(); - } - }).cache(); + JavaDoubleRDD doubles = rdd.mapToDouble(Integer::doubleValue).cache(); doubles.collect(); - JavaPairRDD<Integer, Integer> pairs = rdd.mapToPair( - new PairFunction<Integer, Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> call(Integer x) { - return new Tuple2<>(x, x); - } - }).cache(); + JavaPairRDD<Integer, Integer> pairs = rdd.mapToPair(x -> new Tuple2<>(x, x)).cache(); pairs.collect(); - JavaRDD<String> strings = rdd.map(new Function<Integer, String>() { - @Override - public String call(Integer x) { - return x.toString(); - } - }).cache(); + JavaRDD<String> strings = rdd.map(Object::toString).cache(); strings.collect(); } @@ -887,39 +769,27 @@ public class JavaAPISuite implements Serializable { public void flatMap() { JavaRDD<String> rdd = sc.parallelize(Arrays.asList("Hello World!", "The quick brown fox jumps over the lazy dog.")); - JavaRDD<String> words = rdd.flatMap(new FlatMapFunction<String, String>() { - @Override - public Iterator<String> call(String x) { - return Arrays.asList(x.split(" ")).iterator(); - } - }); + JavaRDD<String> words = rdd.flatMap(x -> Arrays.asList(x.split(" ")).iterator()); assertEquals("Hello", words.first()); assertEquals(11, words.count()); - JavaPairRDD<String, String> pairsRDD = rdd.flatMapToPair( - new PairFlatMapFunction<String, String, String>() { - @Override - public Iterator<Tuple2<String, String>> call(String s) { - List<Tuple2<String, String>> pairs = new LinkedList<>(); - for (String word : s.split(" ")) { - pairs.add(new Tuple2<>(word, word)); - } - return pairs.iterator(); + JavaPairRDD<String, String> pairsRDD = rdd.flatMapToPair(s -> { + List<Tuple2<String, String>> pairs = new LinkedList<>(); + for (String word : s.split(" ")) { + pairs.add(new Tuple2<>(word, word)); } + return pairs.iterator(); } ); assertEquals(new Tuple2<>("Hello", "Hello"), pairsRDD.first()); assertEquals(11, pairsRDD.count()); - JavaDoubleRDD doubles = rdd.flatMapToDouble(new DoubleFlatMapFunction<String>() { - @Override - public Iterator<Double> call(String s) { - List<Double> lengths = new LinkedList<>(); - for (String word : s.split(" ")) { - lengths.add((double) word.length()); - } - return lengths.iterator(); + JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> { + List<Double> lengths = new LinkedList<>(); + for (String word : s.split(" ")) { + lengths.add((double) word.length()); } + return lengths.iterator(); }); assertEquals(5.0, doubles.first(), 0.01); assertEquals(11, pairsRDD.count()); @@ -937,37 +807,23 @@ public class JavaAPISuite implements Serializable { // Regression test for SPARK-668: JavaPairRDD<String, Integer> swapped = pairRDD.flatMapToPair( - new PairFlatMapFunction<Tuple2<Integer, String>, String, Integer>() { - @Override - public Iterator<Tuple2<String, Integer>> call(Tuple2<Integer, String> item) { - return Collections.singletonList(item.swap()).iterator(); - } - }); + item -> Collections.singletonList(item.swap()).iterator()); swapped.collect(); // There was never a bug here, but it's worth testing: - pairRDD.mapToPair(new PairFunction<Tuple2<Integer, String>, String, Integer>() { - @Override - public Tuple2<String, Integer> call(Tuple2<Integer, String> item) { - return item.swap(); - } - }).collect(); + pairRDD.mapToPair(Tuple2::swap).collect(); } @Test public void mapPartitions() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - JavaRDD<Integer> partitionSums = rdd.mapPartitions( - new FlatMapFunction<Iterator<Integer>, Integer>() { - @Override - public Iterator<Integer> call(Iterator<Integer> iter) { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - return Collections.singletonList(sum).iterator(); + JavaRDD<Integer> partitionSums = rdd.mapPartitions(iter -> { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); } - }); + return Collections.singletonList(sum).iterator(); + }); assertEquals("[3, 7]", partitionSums.collect().toString()); } @@ -975,17 +831,13 @@ public class JavaAPISuite implements Serializable { @Test public void mapPartitionsWithIndex() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); - JavaRDD<Integer> partitionSums = rdd.mapPartitionsWithIndex( - new Function2<Integer, Iterator<Integer>, Iterator<Integer>>() { - @Override - public Iterator<Integer> call(Integer index, Iterator<Integer> iter) { - int sum = 0; - while (iter.hasNext()) { - sum += iter.next(); - } - return Collections.singletonList(sum).iterator(); + JavaRDD<Integer> partitionSums = rdd.mapPartitionsWithIndex((index, iter) -> { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); } - }, false); + return Collections.singletonList(sum).iterator(); + }, false); assertEquals("[3, 7]", partitionSums.collect().toString()); } @@ -1124,21 +976,12 @@ public class JavaAPISuite implements Serializable { ); JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs); - rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() { - @Override - public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); // Try reading the output back as an object file JavaPairRDD<Integer, String> readRDD = sc.sequenceFile(outputDir, IntWritable.class, - Text.class).mapToPair(new PairFunction<Tuple2<IntWritable, Text>, Integer, String>() { - @Override - public Tuple2<Integer, String> call(Tuple2<IntWritable, Text> pair) { - return new Tuple2<>(pair._1().get(), pair._2().toString()); - } - }); + Text.class).mapToPair(pair -> new Tuple2<>(pair._1().get(), pair._2().toString())); assertEquals(pairs, readRDD.collect()); } @@ -1179,12 +1022,7 @@ public class JavaAPISuite implements Serializable { channel1.close(); JavaPairRDD<String, PortableDataStream> readRDD = sc.binaryFiles(tempDirName).cache(); - readRDD.foreach(new VoidFunction<Tuple2<String,PortableDataStream>>() { - @Override - public void call(Tuple2<String, PortableDataStream> pair) { - pair._2().toArray(); // force the file to read - } - }); + readRDD.foreach(pair -> pair._2().toArray()); // force the file to read List<Tuple2<String, PortableDataStream>> result = readRDD.collect(); for (Tuple2<String, PortableDataStream> res : result) { @@ -1229,23 +1067,13 @@ public class JavaAPISuite implements Serializable { ); JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs); - rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() { - @Override - public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsNewAPIHadoopFile( - outputDir, IntWritable.class, Text.class, + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsNewAPIHadoopFile(outputDir, IntWritable.class, Text.class, org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.class); 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(); - } - }).collect().toString()); + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); } @SuppressWarnings("unchecked") @@ -1259,22 +1087,13 @@ public class JavaAPISuite implements Serializable { ); JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs); - rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() { - @Override - public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); JavaPairRDD<IntWritable, Text> output = sc.newAPIHadoopFile(outputDir, org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat.class, IntWritable.class, Text.class, Job.getInstance().getConfiguration()); - assertEquals(pairs.toString(), output.map(new Function<Tuple2<IntWritable, Text>, String>() { - @Override - public String call(Tuple2<IntWritable, Text> x) { - return x.toString(); - } - }).collect().toString()); + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); } @Test @@ -1315,21 +1134,12 @@ public class JavaAPISuite implements Serializable { ); JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs); - rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() { - @Override - public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir, SequenceFileInputFormat.class, 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(); - } - }).collect().toString()); + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); } @SuppressWarnings("unchecked") @@ -1343,34 +1153,19 @@ public class JavaAPISuite implements Serializable { ); JavaPairRDD<Integer, String> rdd = sc.parallelizePairs(pairs); - rdd.mapToPair(new PairFunction<Tuple2<Integer, String>, IntWritable, Text>() { - @Override - public Tuple2<IntWritable, Text> call(Tuple2<Integer, String> pair) { - return new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2())); - } - }).saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, - DefaultCodec.class); + rdd.mapToPair(pair -> new Tuple2<>(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class, DefaultCodec.class); JavaPairRDD<IntWritable, Text> output = sc.hadoopFile(outputDir, SequenceFileInputFormat.class, 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(); - } - }).collect().toString()); + assertEquals(pairs.toString(), output.map(Tuple2::toString).collect().toString()); } @Test public void zip() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction<Integer>() { - @Override - public double call(Integer x) { - return x.doubleValue(); - } - }); + JavaDoubleRDD doubles = rdd.mapToDouble(Integer::doubleValue); JavaPairRDD<Integer, Double> zipped = rdd.zip(doubles); zipped.count(); } @@ -1380,12 +1175,7 @@ public class JavaAPISuite implements Serializable { JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); JavaRDD<String> rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer> sizesFn = - new FlatMapFunction2<Iterator<Integer>, Iterator<String>, Integer>() { - @Override - public Iterator<Integer> call(Iterator<Integer> i, Iterator<String> s) { - return Arrays.asList(Iterators.size(i), Iterators.size(s)).iterator(); - } - }; + (i, s) -> Arrays.asList(Iterators.size(i), Iterators.size(s)).iterator(); JavaRDD<Integer> sizes = rdd1.zipPartitions(rdd2, sizesFn); assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); @@ -1396,22 +1186,12 @@ public class JavaAPISuite implements Serializable { public void accumulators() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - final Accumulator<Integer> intAccum = sc.intAccumulator(10); - rdd.foreach(new VoidFunction<Integer>() { - @Override - public void call(Integer x) { - intAccum.add(x); - } - }); + Accumulator<Integer> intAccum = sc.intAccumulator(10); + rdd.foreach(intAccum::add); assertEquals((Integer) 25, intAccum.value()); - final Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0); - rdd.foreach(new VoidFunction<Integer>() { - @Override - public void call(Integer x) { - doubleAccum.add((double) x); - } - }); + Accumulator<Double> doubleAccum = sc.doubleAccumulator(10.0); + rdd.foreach(x -> doubleAccum.add((double) x)); assertEquals((Double) 25.0, doubleAccum.value()); // Try a custom accumulator type @@ -1432,13 +1212,8 @@ public class JavaAPISuite implements Serializable { } }; - final Accumulator<Float> floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); - rdd.foreach(new VoidFunction<Integer>() { - @Override - public void call(Integer x) { - floatAccum.add((float) x); - } - }); + Accumulator<Float> floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); + rdd.foreach(x -> floatAccum.add((float) x)); assertEquals((Float) 25.0f, floatAccum.value()); // Test the setValue method @@ -1449,12 +1224,7 @@ public class JavaAPISuite implements Serializable { @Test public void keyBy() { JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1, 2)); - List<Tuple2<String, Integer>> s = rdd.keyBy(new Function<Integer, String>() { - @Override - public String call(Integer t) { - return t.toString(); - } - }).collect(); + List<Tuple2<String, Integer>> s = rdd.keyBy(Object::toString).collect(); assertEquals(new Tuple2<>("1", 1), s.get(0)); assertEquals(new Tuple2<>("2", 2), s.get(1)); } @@ -1487,26 +1257,10 @@ public class JavaAPISuite implements Serializable { @Test public void combineByKey() { JavaRDD<Integer> originalRDD = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6)); - Function<Integer, Integer> keyFunction = new Function<Integer, Integer>() { - @Override - public Integer call(Integer v1) { - return v1 % 3; - } - }; - Function<Integer, Integer> createCombinerFunction = new Function<Integer, Integer>() { - @Override - public Integer call(Integer v1) { - return v1; - } - }; + Function<Integer, Integer> keyFunction = v1 -> v1 % 3; + Function<Integer, Integer> createCombinerFunction = v1 -> v1; - Function2<Integer, Integer, Integer> mergeValueFunction = - new Function2<Integer, Integer, Integer>() { - @Override - public Integer call(Integer v1, Integer v2) { - return v1 + v2; - } - }; + Function2<Integer, Integer, Integer> mergeValueFunction = (v1, v2) -> v1 + v2; JavaPairRDD<Integer, Integer> combinedRDD = originalRDD.keyBy(keyFunction) .combineByKey(createCombinerFunction, mergeValueFunction, mergeValueFunction); @@ -1534,20 +1288,8 @@ public class JavaAPISuite implements Serializable { @Test public void mapOnPairRDD() { JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); - JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair( - new PairFunction<Integer, Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<>(i, i % 2); - } - }); - JavaPairRDD<Integer, Integer> rdd3 = rdd2.mapToPair( - new PairFunction<Tuple2<Integer, Integer>, Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> call(Tuple2<Integer, Integer> in) { - return new Tuple2<>(in._2(), in._1()); - } - }); + JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); + JavaPairRDD<Integer, Integer> rdd3 = rdd2.mapToPair(in -> new Tuple2<>(in._2(), in._1())); assertEquals(Arrays.asList( new Tuple2<>(1, 1), new Tuple2<>(0, 2), @@ -1561,13 +1303,7 @@ public class JavaAPISuite implements Serializable { public void collectPartitions() { JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); - JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair( - new PairFunction<Integer, Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<>(i, i % 2); - } - }); + JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i, i % 2)); List<Integer>[] parts = rdd1.collectPartitions(new int[] {0}); assertEquals(Arrays.asList(1, 2), parts[0]); @@ -1623,13 +1359,7 @@ public class JavaAPISuite implements Serializable { public void collectAsMapWithIntArrayValues() { // Regression test for SPARK-1040 JavaRDD<Integer> rdd = sc.parallelize(Arrays.asList(1)); - JavaPairRDD<Integer, int[]> pairRDD = rdd.mapToPair( - new PairFunction<Integer, Integer, int[]>() { - @Override - public Tuple2<Integer, int[]> call(Integer x) { - return new Tuple2<>(x, new int[]{x}); - } - }); + JavaPairRDD<Integer, int[]> pairRDD = rdd.mapToPair(x -> new Tuple2<>(x, new int[]{x})); pairRDD.collect(); // Works fine pairRDD.collectAsMap(); // Used to crash with ClassCastException } @@ -1651,13 +1381,7 @@ public class JavaAPISuite implements Serializable { @SuppressWarnings("unchecked") public void sampleByKey() { JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); - JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair( - new PairFunction<Integer, Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<>(i % 2, 1); - } - }); + JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i % 2, 1)); Map<Integer, Double> fractions = new HashMap<>(); fractions.put(0, 0.5); fractions.put(1, 1.0); @@ -1677,13 +1401,7 @@ public class JavaAPISuite implements Serializable { @SuppressWarnings("unchecked") public void sampleByKeyExact() { JavaRDD<Integer> rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); - JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair( - new PairFunction<Integer, Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<>(i % 2, 1); - } - }); + JavaPairRDD<Integer, Integer> rdd2 = rdd1.mapToPair(i -> new Tuple2<>(i % 2, 1)); Map<Integer, Double> fractions = new HashMap<>(); fractions.put(0, 0.5); fractions.put(1, 1.0); @@ -1754,14 +1472,7 @@ public class JavaAPISuite implements Serializable { public void foreachAsync() throws Exception { List<Integer> data = Arrays.asList(1, 2, 3, 4, 5); JavaRDD<Integer> rdd = sc.parallelize(data, 1); - JavaFutureAction<Void> future = rdd.foreachAsync( - new VoidFunction<Integer>() { - @Override - public void call(Integer integer) { - // intentionally left blank. - } - } - ); + JavaFutureAction<Void> future = rdd.foreachAsync(integer -> {}); future.get(); assertFalse(future.isCancelled()); assertTrue(future.isDone()); @@ -1784,11 +1495,8 @@ public class JavaAPISuite implements Serializable { public void testAsyncActionCancellation() throws Exception { List<Integer> data = Arrays.asList(1, 2, 3, 4, 5); JavaRDD<Integer> rdd = sc.parallelize(data, 1); - JavaFutureAction<Void> future = rdd.foreachAsync(new VoidFunction<Integer>() { - @Override - public void call(Integer integer) throws InterruptedException { - Thread.sleep(10000); // To ensure that the job won't finish before it's cancelled. - } + JavaFutureAction<Void> future = rdd.foreachAsync(integer -> { + Thread.sleep(10000); // To ensure that the job won't finish before it's cancelled. }); future.cancel(true); assertTrue(future.isCancelled()); @@ -1805,7 +1513,7 @@ public class JavaAPISuite implements Serializable { public void testAsyncActionErrorWrapping() throws Exception { List<Integer> data = Arrays.asList(1, 2, 3, 4, 5); JavaRDD<Integer> rdd = sc.parallelize(data, 1); - JavaFutureAction<Long> future = rdd.map(new BuggyMapFunction<Integer>()).countAsync(); + JavaFutureAction<Long> future = rdd.map(new BuggyMapFunction<>()).countAsync(); try { future.get(2, TimeUnit.SECONDS); fail("Expected future.get() for failed job to throw ExcecutionException"); diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java index ba57b6beb2..938cc8ddfb 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -59,39 +59,39 @@ public class JavaConsumerStrategySuite implements Serializable { ); final ConsumerStrategy<String, String> sub1 = - ConsumerStrategies.<String, String>Subscribe(sTopics, sKafkaParams, sOffsets); + ConsumerStrategies.Subscribe(sTopics, sKafkaParams, sOffsets); final ConsumerStrategy<String, String> sub2 = - ConsumerStrategies.<String, String>Subscribe(sTopics, sKafkaParams); + ConsumerStrategies.Subscribe(sTopics, sKafkaParams); final ConsumerStrategy<String, String> sub3 = - ConsumerStrategies.<String, String>Subscribe(topics, kafkaParams, offsets); + ConsumerStrategies.Subscribe(topics, kafkaParams, offsets); final ConsumerStrategy<String, String> sub4 = - ConsumerStrategies.<String, String>Subscribe(topics, kafkaParams); + ConsumerStrategies.Subscribe(topics, kafkaParams); Assert.assertEquals( sub1.executorKafkaParams().get("bootstrap.servers"), sub3.executorKafkaParams().get("bootstrap.servers")); final ConsumerStrategy<String, String> psub1 = - ConsumerStrategies.<String, String>SubscribePattern(pat, sKafkaParams, sOffsets); + ConsumerStrategies.SubscribePattern(pat, sKafkaParams, sOffsets); final ConsumerStrategy<String, String> psub2 = - ConsumerStrategies.<String, String>SubscribePattern(pat, sKafkaParams); + ConsumerStrategies.SubscribePattern(pat, sKafkaParams); final ConsumerStrategy<String, String> psub3 = - ConsumerStrategies.<String, String>SubscribePattern(pat, kafkaParams, offsets); + ConsumerStrategies.SubscribePattern(pat, kafkaParams, offsets); final ConsumerStrategy<String, String> psub4 = - ConsumerStrategies.<String, String>SubscribePattern(pat, kafkaParams); + ConsumerStrategies.SubscribePattern(pat, kafkaParams); Assert.assertEquals( psub1.executorKafkaParams().get("bootstrap.servers"), psub3.executorKafkaParams().get("bootstrap.servers")); final ConsumerStrategy<String, String> asn1 = - ConsumerStrategies.<String, String>Assign(sParts, sKafkaParams, sOffsets); + ConsumerStrategies.Assign(sParts, sKafkaParams, sOffsets); final ConsumerStrategy<String, String> asn2 = - ConsumerStrategies.<String, String>Assign(sParts, sKafkaParams); + ConsumerStrategies.Assign(sParts, sKafkaParams); final ConsumerStrategy<String, String> asn3 = - ConsumerStrategies.<String, String>Assign(parts, kafkaParams, offsets); + ConsumerStrategies.Assign(parts, kafkaParams, offsets); final ConsumerStrategy<String, String> asn4 = - ConsumerStrategies.<String, String>Assign(parts, kafkaParams); + ConsumerStrategies.Assign(parts, kafkaParams); Assert.assertEquals( asn1.executorKafkaParams().get("bootstrap.servers"), diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index d569b6688d..2e050f8413 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -217,7 +217,7 @@ public class SparkSubmitCommandBuilderSuite extends BaseSuite { String deployMode = isDriver ? "client" : "cluster"; SparkSubmitCommandBuilder launcher = - newCommandBuilder(Collections.<String>emptyList()); + newCommandBuilder(Collections.emptyList()); launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); launcher.master = "yarn"; diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java index 3bc35da7cc..9ff7aceb58 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitOptionParserSuite.java @@ -44,7 +44,7 @@ public class SparkSubmitOptionParserSuite extends BaseSuite { count++; verify(parser).handle(eq(optNames[0]), eq(value)); verify(parser, times(count)).handle(anyString(), anyString()); - verify(parser, times(count)).handleExtraArgs(eq(Collections.<String>emptyList())); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); } } @@ -54,9 +54,9 @@ public class SparkSubmitOptionParserSuite extends BaseSuite { parser.parse(Arrays.asList(name)); count++; switchCount++; - verify(parser, times(switchCount)).handle(eq(switchNames[0]), same((String) null)); + verify(parser, times(switchCount)).handle(eq(switchNames[0]), same(null)); verify(parser, times(count)).handle(anyString(), any(String.class)); - verify(parser, times(count)).handleExtraArgs(eq(Collections.<String>emptyList())); + verify(parser, times(count)).handleExtraArgs(eq(Collections.emptyList())); } } } @@ -80,7 +80,7 @@ public class SparkSubmitOptionParserSuite extends BaseSuite { List<String> args = Arrays.asList(parser.MASTER + "=" + parser.MASTER); parser.parse(args); verify(parser).handle(eq(parser.MASTER), eq(parser.MASTER)); - verify(parser).handleExtraArgs(eq(Collections.<String>emptyList())); + verify(parser).handleExtraArgs(eq(Collections.emptyList())); } private static class DummyParser extends SparkSubmitOptionParser { diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index 8c0338e284..683ceffeae 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -21,16 +21,14 @@ import java.io.Serializable; import java.util.Arrays; import java.util.List; -import scala.Tuple2; - import org.junit.Assert; import org.junit.Test; import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.Vectors; +import org.apache.spark.mllib.linalg.DenseVector; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; @@ -69,35 +67,22 @@ public class JavaPCASuite extends SharedSparkSession { JavaRDD<Vector> dataRDD = jsc.parallelize(points, 2); RowMatrix mat = new RowMatrix(dataRDD.map( - new Function<Vector, org.apache.spark.mllib.linalg.Vector>() { - public org.apache.spark.mllib.linalg.Vector call(Vector vector) { - return new org.apache.spark.mllib.linalg.DenseVector(vector.toArray()); - } - } + (Vector vector) -> (org.apache.spark.mllib.linalg.Vector) new DenseVector(vector.toArray()) ).rdd()); Matrix pc = mat.computePrincipalComponents(3); mat.multiply(pc).rows().toJavaRDD(); - JavaRDD<Vector> expected = mat.multiply(pc).rows().toJavaRDD().map( - new Function<org.apache.spark.mllib.linalg.Vector, Vector>() { - public Vector call(org.apache.spark.mllib.linalg.Vector vector) { - return vector.asML(); - } - } - ); + JavaRDD<Vector> expected = mat.multiply(pc).rows().toJavaRDD() + .map(org.apache.spark.mllib.linalg.Vector::asML); - JavaRDD<VectorPair> featuresExpected = dataRDD.zip(expected).map( - new Function<Tuple2<Vector, Vector>, VectorPair>() { - public VectorPair call(Tuple2<Vector, Vector> pair) { - VectorPair featuresExpected = new VectorPair(); - featuresExpected.setFeatures(pair._1()); - featuresExpected.setExpected(pair._2()); - return featuresExpected; - } - } - ); + JavaRDD<VectorPair> featuresExpected = dataRDD.zip(expected).map(pair -> { + VectorPair featuresExpected1 = new VectorPair(); + featuresExpected1.setFeatures(pair._1()); + featuresExpected1.setExpected(pair._2()); + return featuresExpected1; + }); Dataset<Row> df = spark.createDataFrame(featuresExpected, VectorPair.class); PCAModel pca = new PCA() diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 6ded42e928..65db3d014f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -25,7 +25,6 @@ import org.junit.Test; import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; @@ -42,7 +41,7 @@ public class JavaNaiveBayesSuite extends SharedSparkSession { new LabeledPoint(2, Vectors.dense(0.0, 0.0, 2.0)) ); - private int validatePrediction(List<LabeledPoint> points, NaiveBayesModel model) { + private static int validatePrediction(List<LabeledPoint> points, NaiveBayesModel model) { int correct = 0; for (LabeledPoint p : points) { if (model.predict(p.features()) == p.label()) { @@ -80,12 +79,7 @@ public class JavaNaiveBayesSuite extends SharedSparkSession { public void testPredictJavaRDD() { JavaRDD<LabeledPoint> examples = jsc.parallelize(POINTS, 2).cache(); NaiveBayesModel model = NaiveBayes.train(examples.rdd()); - JavaRDD<Vector> vectors = examples.map(new Function<LabeledPoint, Vector>() { - @Override - public Vector call(LabeledPoint v) throws Exception { - return v.features(); - } - }); + JavaRDD<Vector> vectors = examples.map(LabeledPoint::features); JavaRDD<Double> predictions = model.predict(vectors); // Should be able to get the first prediction. predictions.first(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java index 3d62b273d2..b4196c6ecd 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaBisectingKMeansSuite.java @@ -17,7 +17,7 @@ package org.apache.spark.mllib.clustering; -import com.google.common.collect.Lists; +import java.util.Arrays; import org.junit.Assert; import org.junit.Test; @@ -31,7 +31,7 @@ public class JavaBisectingKMeansSuite extends SharedSparkSession { @Test public void twoDimensionalData() { - JavaRDD<Vector> points = jsc.parallelize(Lists.newArrayList( + JavaRDD<Vector> points = jsc.parallelize(Arrays.asList( Vectors.dense(4, -1), Vectors.dense(4, 1), Vectors.sparse(2, new int[]{0}, new double[]{1.0}) diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java index 08d6713ab2..38ee2507f2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java @@ -20,6 +20,7 @@ package org.apache.spark.mllib.clustering; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import scala.Tuple2; import scala.Tuple3; @@ -30,7 +31,6 @@ import static org.junit.Assert.*; import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; @@ -39,7 +39,7 @@ public class JavaLDASuite extends SharedSparkSession { @Override public void setUp() throws IOException { super.setUp(); - ArrayList<Tuple2<Long, Vector>> tinyCorpus = new ArrayList<>(); + List<Tuple2<Long, Vector>> tinyCorpus = new ArrayList<>(); for (int i = 0; i < LDASuite.tinyCorpus().length; i++) { tinyCorpus.add(new Tuple2<>((Long) LDASuite.tinyCorpus()[i]._1(), LDASuite.tinyCorpus()[i]._2())); @@ -53,7 +53,7 @@ public class JavaLDASuite extends SharedSparkSession { Matrix topics = LDASuite.tinyTopics(); double[] topicConcentration = new double[topics.numRows()]; Arrays.fill(topicConcentration, 1.0D / topics.numRows()); - LocalLDAModel model = new LocalLDAModel(topics, Vectors.dense(topicConcentration), 1D, 100D); + LocalLDAModel model = new LocalLDAModel(topics, Vectors.dense(topicConcentration), 1.0, 100.0); // Check: basic parameters assertEquals(model.k(), tinyK); @@ -87,17 +87,17 @@ public class JavaLDASuite extends SharedSparkSession { // Check: basic parameters LocalLDAModel localModel = model.toLocal(); - assertEquals(model.k(), k); - assertEquals(localModel.k(), k); - assertEquals(model.vocabSize(), tinyVocabSize); - assertEquals(localModel.vocabSize(), tinyVocabSize); - assertEquals(model.topicsMatrix(), localModel.topicsMatrix()); + assertEquals(k, model.k()); + assertEquals(k, localModel.k()); + assertEquals(tinyVocabSize, model.vocabSize()); + assertEquals(tinyVocabSize, localModel.vocabSize()); + assertEquals(localModel.topicsMatrix(), model.topicsMatrix()); // Check: topic summaries Tuple2<int[], double[]>[] roundedTopicSummary = model.describeTopics(); - assertEquals(roundedTopicSummary.length, k); + assertEquals(k, roundedTopicSummary.length); Tuple2<int[], double[]>[] roundedLocalTopicSummary = localModel.describeTopics(); - assertEquals(roundedLocalTopicSummary.length, k); + assertEquals(k, roundedLocalTopicSummary.length); // Check: log probabilities assertTrue(model.logLikelihood() < 0.0); @@ -107,12 +107,8 @@ public class JavaLDASuite extends SharedSparkSession { JavaPairRDD<Long, Vector> topicDistributions = model.javaTopicDistributions(); // SPARK-5562. since the topicDistribution returns the distribution of the non empty docs // over topics. Compare it against nonEmptyCorpus instead of corpus - JavaPairRDD<Long, Vector> nonEmptyCorpus = corpus.filter( - new Function<Tuple2<Long, Vector>, Boolean>() { - public Boolean call(Tuple2<Long, Vector> tuple2) { - return Vectors.norm(tuple2._2(), 1.0) != 0.0; - } - }); + JavaPairRDD<Long, Vector> nonEmptyCorpus = + corpus.filter(tuple2 -> Vectors.norm(tuple2._2(), 1.0) != 0.0); assertEquals(topicDistributions.count(), nonEmptyCorpus.count()); // Check: javaTopTopicsPerDocuments @@ -155,14 +151,14 @@ public class JavaLDASuite extends SharedSparkSession { LDAModel model = lda.run(corpus); // Check: basic parameters - assertEquals(model.k(), k); - assertEquals(model.vocabSize(), tinyVocabSize); + assertEquals(k, model.k()); + assertEquals(tinyVocabSize, model.vocabSize()); // Check: topic summaries Tuple2<int[], double[]>[] roundedTopicSummary = model.describeTopics(); - assertEquals(roundedTopicSummary.length, k); + assertEquals(k, roundedTopicSummary.length); Tuple2<int[], double[]>[] roundedLocalTopicSummary = model.describeTopics(); - assertEquals(roundedLocalTopicSummary.length, k); + assertEquals(k, roundedLocalTopicSummary.length); } @Test @@ -177,7 +173,7 @@ public class JavaLDASuite extends SharedSparkSession { double logPerplexity = toyModel.logPerplexity(pairedDocs); // check: logLikelihood. - ArrayList<Tuple2<Long, Vector>> docsSingleWord = new ArrayList<>(); + List<Tuple2<Long, Vector>> docsSingleWord = new ArrayList<>(); docsSingleWord.add(new Tuple2<>(0L, Vectors.dense(1.0, 0.0, 0.0))); JavaPairRDD<Long, Vector> single = JavaPairRDD.fromJavaRDD(jsc.parallelize(docsSingleWord)); double logLikelihood = toyModel.logLikelihood(single); @@ -190,6 +186,6 @@ public class JavaLDASuite extends SharedSparkSession { LDASuite.tinyTopicDescription(); private JavaPairRDD<Long, Vector> corpus; private LocalLDAModel toyModel = LDASuite.toyModel(); - private ArrayList<Tuple2<Long, Vector>> toyData = LDASuite.javaToyData(); + private List<Tuple2<Long, Vector>> toyData = LDASuite.javaToyData(); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java index 3451e07737..15de566c88 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java @@ -31,9 +31,9 @@ public class JavaAssociationRulesSuite extends SharedSparkSession { @SuppressWarnings("unchecked") JavaRDD<FPGrowth.FreqItemset<String>> freqItemsets = jsc.parallelize(Arrays.asList( - new FreqItemset<String>(new String[]{"a"}, 15L), - new FreqItemset<String>(new String[]{"b"}, 35L), - new FreqItemset<String>(new String[]{"a", "b"}, 12L) + new FreqItemset<>(new String[]{"a"}, 15L), + new FreqItemset<>(new String[]{"b"}, 35L), + new FreqItemset<>(new String[]{"a", "b"}, 12L) )); JavaRDD<AssociationRules.Rule<String>> results = (new AssociationRules()).run(freqItemsets); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index a46b1321b3..86c723aa00 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -24,13 +24,13 @@ import org.junit.Test; import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.util.LinearDataGenerator; public class JavaLinearRegressionSuite extends SharedSparkSession { - int validatePrediction(List<LabeledPoint> validationData, LinearRegressionModel model) { + private static int validatePrediction( + List<LabeledPoint> validationData, LinearRegressionModel model) { int numAccurate = 0; for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); @@ -87,12 +87,7 @@ public class JavaLinearRegressionSuite extends SharedSparkSession { LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 42, 0.1), 2).cache(); LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); - JavaRDD<Vector> vectors = testRDD.map(new Function<LabeledPoint, Vector>() { - @Override - public Vector call(LabeledPoint v) throws Exception { - return v.features(); - } - }); + JavaRDD<Vector> vectors = testRDD.map(LabeledPoint::features); JavaRDD<Double> predictions = model.predict(vectors); // Should be able to get the first prediction. predictions.first(); diff --git a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java index 1dcbbcaa02..0f71deb9ea 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java @@ -25,8 +25,6 @@ import org.junit.Test; import org.apache.spark.SharedSparkSession; import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.tree.configuration.Algo; import org.apache.spark.mllib.tree.configuration.Strategy; @@ -35,7 +33,7 @@ import org.apache.spark.mllib.tree.model.DecisionTreeModel; public class JavaDecisionTreeSuite extends SharedSparkSession { - int validatePrediction(List<LabeledPoint> validationData, DecisionTreeModel model) { + private static int validatePrediction(List<LabeledPoint> validationData, DecisionTreeModel model) { int numCorrect = 0; for (LabeledPoint point : validationData) { Double prediction = model.predict(point.features()); @@ -63,7 +61,7 @@ public class JavaDecisionTreeSuite extends SharedSparkSession { DecisionTreeModel model = learner.run(rdd.rdd()); int numCorrect = validatePrediction(arr, model); - Assert.assertTrue(numCorrect == rdd.count()); + Assert.assertEquals(numCorrect, rdd.count()); } @Test @@ -82,15 +80,10 @@ public class JavaDecisionTreeSuite extends SharedSparkSession { DecisionTreeModel model = DecisionTree$.MODULE$.train(rdd.rdd(), strategy); // java compatibility test - JavaRDD<Double> predictions = model.predict(rdd.map(new Function<LabeledPoint, Vector>() { - @Override - public Vector call(LabeledPoint v1) { - return v1.features(); - } - })); + JavaRDD<Double> predictions = model.predict(rdd.map(LabeledPoint::features)); int numCorrect = validatePrediction(arr, model); - Assert.assertTrue(numCorrect == rdd.count()); + Assert.assertEquals(numCorrect, rdd.count()); } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index 06cd9ea2d2..bf87174835 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -157,7 +157,7 @@ public abstract class SpecificParquetRecordReaderBase<T> extends RecordReader<Vo // to the accumulator. So we can check if the row groups are filtered or not in test case. TaskContext taskContext = TaskContext$.MODULE$.get(); if (taskContext != null) { - Option<AccumulatorV2<?, ?>> accu = (Option<AccumulatorV2<?, ?>>) taskContext.taskMetrics() + Option<AccumulatorV2<?, ?>> accu = taskContext.taskMetrics() .lookForAccumulatorByName("numRowGroups"); if (accu.isDefined()) { ((LongAccumulator)accu.get()).add((long)blocks.size()); diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java index 8b8a403e2b..6ffccee52c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/Java8DatasetAggregatorSuite.java @@ -35,27 +35,35 @@ public class Java8DatasetAggregatorSuite extends JavaDatasetAggregatorSuiteBase public void testTypedAggregationAverage() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.avg(v -> (double)(v._2() * 2))); - Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 6.0)), agged.collectAsList()); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 6.0)), + agged.collectAsList()); } @Test public void testTypedAggregationCount() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.count(v -> v)); - Assert.assertEquals(Arrays.asList(tuple2("a", 2L), tuple2("b", 1L)), agged.collectAsList()); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 2L), new Tuple2<>("b", 1L)), + agged.collectAsList()); } @Test public void testTypedAggregationSumDouble() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.sum(v -> (double)v._2())); - Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 3.0)), agged.collectAsList()); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 3.0)), + agged.collectAsList()); } @Test public void testTypedAggregationSumLong() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.sumLong(v -> (long)v._2())); - Assert.assertEquals(Arrays.asList(tuple2("a", 3L), tuple2("b", 3L)), agged.collectAsList()); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3L), new Tuple2<>("b", 3L)), + agged.collectAsList()); } } 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 573d0e3594..bf8ff61eae 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 @@ -30,7 +30,6 @@ import org.junit.Test; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -95,12 +94,7 @@ public class JavaApplySchemaSuite implements Serializable { personList.add(person2); JavaRDD<Row> rowRDD = jsc.parallelize(personList).map( - new Function<Person, Row>() { - @Override - public Row call(Person person) throws Exception { - return RowFactory.create(person.getName(), person.getAge()); - } - }); + person -> RowFactory.create(person.getName(), person.getAge())); List<StructField> fields = new ArrayList<>(2); fields.add(DataTypes.createStructField("name", DataTypes.StringType, false)); @@ -131,12 +125,7 @@ public class JavaApplySchemaSuite implements Serializable { personList.add(person2); JavaRDD<Row> rowRDD = jsc.parallelize(personList).map( - new Function<Person, Row>() { - @Override - public Row call(Person person) { - return RowFactory.create(person.getName(), person.getAge()); - } - }); + person -> RowFactory.create(person.getName(), person.getAge())); List<StructField> fields = new ArrayList<>(2); fields.add(DataTypes.createStructField("", DataTypes.StringType, false)); @@ -146,12 +135,7 @@ public class JavaApplySchemaSuite implements Serializable { Dataset<Row> df = spark.createDataFrame(rowRDD, schema); df.createOrReplaceTempView("people"); List<String> actual = spark.sql("SELECT * FROM people").toJavaRDD() - .map(new Function<Row, String>() { - @Override - public String call(Row row) { - return row.getString(0) + "_" + row.get(1); - } - }).collect(); + .map(row -> 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 c44fc3d393..c3b94a44c2 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 @@ -189,7 +189,7 @@ public class JavaDataFrameSuite { for (int i = 0; i < d.length(); i++) { Assert.assertEquals(bean.getD().get(i), d.apply(i)); } - // Java.math.BigInteger is equavient to Spark Decimal(38,0) + // Java.math.BigInteger is equivalent to Spark Decimal(38,0) Assert.assertEquals(new BigDecimal(bean.getE()), first.getDecimal(4)); } @@ -231,13 +231,10 @@ public class JavaDataFrameSuite { Assert.assertEquals(0, schema2.fieldIndex("id")); } - private static final Comparator<Row> crosstabRowComparator = new Comparator<Row>() { - @Override - public int compare(Row row1, Row row2) { - String item1 = row1.getString(0); - String item2 = row2.getString(0); - return item1.compareTo(item2); - } + private static final Comparator<Row> crosstabRowComparator = (row1, row2) -> { + String item1 = row1.getString(0); + String item2 = row2.getString(0); + return item1.compareTo(item2); }; @Test @@ -249,7 +246,7 @@ public class JavaDataFrameSuite { Assert.assertEquals("1", columnNames[1]); Assert.assertEquals("2", columnNames[2]); List<Row> rows = crosstab.collectAsList(); - Collections.sort(rows, crosstabRowComparator); + rows.sort(crosstabRowComparator); Integer count = 1; for (Row row : rows) { Assert.assertEquals(row.get(0).toString(), count.toString()); @@ -284,7 +281,7 @@ public class JavaDataFrameSuite { @Test public void testSampleBy() { Dataset<Row> df = spark.range(0, 100, 1, 2).select(col("id").mod(3).as("key")); - Dataset<Row> sampled = df.stat().<Integer>sampleBy("key", ImmutableMap.of(0, 0.1, 1, 0.2), 0L); + Dataset<Row> sampled = df.stat().sampleBy("key", ImmutableMap.of(0, 0.1, 1, 0.2), 0L); List<Row> actual = sampled.groupBy("key").count().orderBy("key").collectAsList(); Assert.assertEquals(0, actual.get(0).getLong(0)); Assert.assertTrue(0 <= actual.get(0).getLong(1) && actual.get(0).getLong(1) <= 8); @@ -296,7 +293,7 @@ public class JavaDataFrameSuite { public void pivot() { Dataset<Row> df = spark.table("courseSales"); List<Row> actual = df.groupBy("year") - .pivot("course", Arrays.<Object>asList("dotNET", "Java")) + .pivot("course", Arrays.asList("dotNET", "Java")) .agg(sum("earnings")).orderBy("year").collectAsList(); Assert.assertEquals(2012, actual.get(0).getInt(0)); @@ -352,24 +349,24 @@ public class JavaDataFrameSuite { Dataset<Long> df = spark.range(1000); CountMinSketch sketch1 = df.stat().countMinSketch("id", 10, 20, 42); - Assert.assertEquals(sketch1.totalCount(), 1000); - Assert.assertEquals(sketch1.depth(), 10); - Assert.assertEquals(sketch1.width(), 20); + Assert.assertEquals(1000, sketch1.totalCount()); + Assert.assertEquals(10, sketch1.depth()); + Assert.assertEquals(20, sketch1.width()); CountMinSketch sketch2 = df.stat().countMinSketch(col("id"), 10, 20, 42); - Assert.assertEquals(sketch2.totalCount(), 1000); - Assert.assertEquals(sketch2.depth(), 10); - Assert.assertEquals(sketch2.width(), 20); + Assert.assertEquals(1000, sketch2.totalCount()); + Assert.assertEquals(10, sketch2.depth()); + Assert.assertEquals(20, sketch2.width()); CountMinSketch sketch3 = df.stat().countMinSketch("id", 0.001, 0.99, 42); - Assert.assertEquals(sketch3.totalCount(), 1000); - Assert.assertEquals(sketch3.relativeError(), 0.001, 1e-4); - Assert.assertEquals(sketch3.confidence(), 0.99, 5e-3); + Assert.assertEquals(1000, sketch3.totalCount()); + Assert.assertEquals(0.001, sketch3.relativeError(), 1.0e-4); + Assert.assertEquals(0.99, sketch3.confidence(), 5.0e-3); CountMinSketch sketch4 = df.stat().countMinSketch(col("id"), 0.001, 0.99, 42); - Assert.assertEquals(sketch4.totalCount(), 1000); - Assert.assertEquals(sketch4.relativeError(), 0.001, 1e-4); - Assert.assertEquals(sketch4.confidence(), 0.99, 5e-3); + Assert.assertEquals(1000, sketch4.totalCount()); + Assert.assertEquals(0.001, sketch4.relativeError(), 1.0e-4); + Assert.assertEquals(0.99, sketch4.confidence(), 5.0e-3); } @Test @@ -389,13 +386,13 @@ public class JavaDataFrameSuite { } BloomFilter filter3 = df.stat().bloomFilter("id", 1000, 64 * 5); - Assert.assertTrue(filter3.bitSize() == 64 * 5); + Assert.assertEquals(64 * 5, filter3.bitSize()); for (int i = 0; i < 1000; i++) { Assert.assertTrue(filter3.mightContain(i)); } BloomFilter filter4 = df.stat().bloomFilter(col("id").multiply(3), 1000, 64 * 5); - Assert.assertTrue(filter4.bitSize() == 64 * 5); + Assert.assertEquals(64 * 5, filter4.bitSize()); for (int i = 0; i < 1000; i++) { Assert.assertTrue(filter4.mightContain(i * 3)); } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java index fe86371516..d3769a74b9 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java @@ -24,7 +24,6 @@ import scala.Tuple2; import org.junit.Assert; import org.junit.Test; -import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.Encoders; @@ -41,7 +40,9 @@ public class JavaDatasetAggregatorSuite extends JavaDatasetAggregatorSuiteBase { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); Dataset<Tuple2<String, Integer>> agged = grouped.agg(new IntSumOf().toColumn()); - Assert.assertEquals(Arrays.asList(tuple2("a", 3), tuple2("b", 3)), agged.collectAsList()); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3), new Tuple2<>("b", 3)), + agged.collectAsList()); Dataset<Tuple2<String, Integer>> agged2 = grouped.agg(new IntSumOf().toColumn()) .as(Encoders.tuple(Encoders.STRING(), Encoders.INT())); @@ -87,48 +88,36 @@ public class JavaDatasetAggregatorSuite extends JavaDatasetAggregatorSuiteBase { @Test public void testTypedAggregationAverage() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); - Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.avg( - new MapFunction<Tuple2<String, Integer>, Double>() { - public Double call(Tuple2<String, Integer> value) throws Exception { - return (double)(value._2() * 2); - } - })); - Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 6.0)), agged.collectAsList()); + Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.avg(value -> (double)(value._2() * 2))); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 6.0)), + agged.collectAsList()); } @Test public void testTypedAggregationCount() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); - Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.count( - new MapFunction<Tuple2<String, Integer>, Object>() { - public Object call(Tuple2<String, Integer> value) throws Exception { - return value; - } - })); - Assert.assertEquals(Arrays.asList(tuple2("a", 2), tuple2("b", 1)), agged.collectAsList()); + Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.count(value -> value)); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 2L), new Tuple2<>("b", 1L)), + agged.collectAsList()); } @Test public void testTypedAggregationSumDouble() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); - Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.sum( - new MapFunction<Tuple2<String, Integer>, Double>() { - public Double call(Tuple2<String, Integer> value) throws Exception { - return (double)value._2(); - } - })); - Assert.assertEquals(Arrays.asList(tuple2("a", 3.0), tuple2("b", 3.0)), agged.collectAsList()); + Dataset<Tuple2<String, Double>> agged = grouped.agg(typed.sum(value -> (double) value._2())); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3.0), new Tuple2<>("b", 3.0)), + agged.collectAsList()); } @Test public void testTypedAggregationSumLong() { KeyValueGroupedDataset<String, Tuple2<String, Integer>> grouped = generateGroupedDataset(); - Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.sumLong( - new MapFunction<Tuple2<String, Integer>, Long>() { - public Long call(Tuple2<String, Integer> value) throws Exception { - return (long)value._2(); - } - })); - Assert.assertEquals(Arrays.asList(tuple2("a", 3), tuple2("b", 3)), agged.collectAsList()); + Dataset<Tuple2<String, Long>> agged = grouped.agg(typed.sumLong(value -> (long) value._2())); + Assert.assertEquals( + Arrays.asList(new Tuple2<>("a", 3L), new Tuple2<>("b", 3L)), + agged.collectAsList()); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java index 8fc4eff55d..e62db7d2cf 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuiteBase.java @@ -52,23 +52,13 @@ public class JavaDatasetAggregatorSuiteBase implements Serializable { spark = null; } - protected <T1, T2> Tuple2<T1, T2> tuple2(T1 t1, T2 t2) { - return new Tuple2<>(t1, t2); - } - protected KeyValueGroupedDataset<String, Tuple2<String, Integer>> generateGroupedDataset() { Encoder<Tuple2<String, Integer>> encoder = Encoders.tuple(Encoders.STRING(), Encoders.INT()); List<Tuple2<String, Integer>> data = - Arrays.asList(tuple2("a", 1), tuple2("a", 2), tuple2("b", 3)); + Arrays.asList(new Tuple2<>("a", 1), new Tuple2<>("a", 2), new Tuple2<>("b", 3)); Dataset<Tuple2<String, Integer>> ds = spark.createDataset(data, encoder); - return ds.groupByKey( - new MapFunction<Tuple2<String, Integer>, String>() { - @Override - public String call(Tuple2<String, Integer> value) throws Exception { - return value._1(); - } - }, + return ds.groupByKey((MapFunction<Tuple2<String, Integer>, String>) value -> value._1(), Encoders.STRING()); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index a94a37cb21..577672ca8e 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -96,12 +96,7 @@ public class JavaDatasetSuite implements Serializable { @Test public void testTypedFilterPreservingSchema() { Dataset<Long> ds = spark.range(10); - Dataset<Long> ds2 = ds.filter(new FilterFunction<Long>() { - @Override - public boolean call(Long value) throws Exception { - return value > 3; - } - }); + Dataset<Long> ds2 = ds.filter((FilterFunction<Long>) value -> value > 3); Assert.assertEquals(ds.schema(), ds2.schema()); } @@ -111,44 +106,28 @@ public class JavaDatasetSuite implements Serializable { Dataset<String> ds = spark.createDataset(data, Encoders.STRING()); Assert.assertEquals("hello", ds.first()); - Dataset<String> filtered = ds.filter(new FilterFunction<String>() { - @Override - public boolean call(String v) throws Exception { - return v.startsWith("h"); - } - }); + Dataset<String> filtered = ds.filter((FilterFunction<String>) v -> v.startsWith("h")); Assert.assertEquals(Arrays.asList("hello"), filtered.collectAsList()); - Dataset<Integer> mapped = ds.map(new MapFunction<String, Integer>() { - @Override - public Integer call(String v) throws Exception { - return v.length(); - } - }, Encoders.INT()); + Dataset<Integer> mapped = ds.map((MapFunction<String, Integer>) v -> v.length(), Encoders.INT()); Assert.assertEquals(Arrays.asList(5, 5), mapped.collectAsList()); - Dataset<String> parMapped = ds.mapPartitions(new MapPartitionsFunction<String, String>() { - @Override - public Iterator<String> call(Iterator<String> it) { - List<String> ls = new LinkedList<>(); - while (it.hasNext()) { - ls.add(it.next().toUpperCase(Locale.ENGLISH)); - } - return ls.iterator(); + Dataset<String> parMapped = ds.mapPartitions((MapPartitionsFunction<String, String>) it -> { + List<String> ls = new LinkedList<>(); + while (it.hasNext()) { + ls.add(it.next().toUpperCase(Locale.ENGLISH)); } + return ls.iterator(); }, Encoders.STRING()); Assert.assertEquals(Arrays.asList("HELLO", "WORLD"), parMapped.collectAsList()); - Dataset<String> flatMapped = ds.flatMap(new FlatMapFunction<String, String>() { - @Override - public Iterator<String> call(String s) { - List<String> ls = new LinkedList<>(); - for (char c : s.toCharArray()) { - ls.add(String.valueOf(c)); - } - return ls.iterator(); + Dataset<String> flatMapped = ds.flatMap((FlatMapFunction<String, String>) s -> { + List<String> ls = new LinkedList<>(); + for (char c : s.toCharArray()) { + ls.add(String.valueOf(c)); } + return ls.iterator(); }, Encoders.STRING()); Assert.assertEquals( Arrays.asList("h", "e", "l", "l", "o", "w", "o", "r", "l", "d"), @@ -157,16 +136,11 @@ public class JavaDatasetSuite implements Serializable { @Test public void testForeach() { - final LongAccumulator accum = jsc.sc().longAccumulator(); + LongAccumulator accum = jsc.sc().longAccumulator(); List<String> data = Arrays.asList("a", "b", "c"); Dataset<String> ds = spark.createDataset(data, Encoders.STRING()); - ds.foreach(new ForeachFunction<String>() { - @Override - public void call(String s) throws Exception { - accum.add(1); - } - }); + ds.foreach((ForeachFunction<String>) s -> accum.add(1)); Assert.assertEquals(3, accum.value().intValue()); } @@ -175,12 +149,7 @@ public class JavaDatasetSuite implements Serializable { List<Integer> data = Arrays.asList(1, 2, 3); Dataset<Integer> ds = spark.createDataset(data, Encoders.INT()); - int reduced = ds.reduce(new ReduceFunction<Integer>() { - @Override - public Integer call(Integer v1, Integer v2) throws Exception { - return v1 + v2; - } - }); + int reduced = ds.reduce((ReduceFunction<Integer>) (v1, v2) -> v1 + v2); Assert.assertEquals(6, reduced); } @@ -189,52 +158,38 @@ public class JavaDatasetSuite implements Serializable { List<String> data = Arrays.asList("a", "foo", "bar"); Dataset<String> ds = spark.createDataset(data, Encoders.STRING()); KeyValueGroupedDataset<Integer, String> grouped = ds.groupByKey( - new MapFunction<String, Integer>() { - @Override - public Integer call(String v) throws Exception { - return v.length(); - } - }, + (MapFunction<String, Integer>) v -> v.length(), Encoders.INT()); - Dataset<String> mapped = grouped.mapGroups(new MapGroupsFunction<Integer, String, String>() { - @Override - public String call(Integer key, Iterator<String> values) throws Exception { - StringBuilder sb = new StringBuilder(key.toString()); - while (values.hasNext()) { - sb.append(values.next()); - } - return sb.toString(); + Dataset<String> mapped = grouped.mapGroups((MapGroupsFunction<Integer, String, String>) (key, values) -> { + StringBuilder sb = new StringBuilder(key.toString()); + while (values.hasNext()) { + sb.append(values.next()); } + return sb.toString(); }, Encoders.STRING()); Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped.collectAsList())); Dataset<String> flatMapped = grouped.flatMapGroups( - new FlatMapGroupsFunction<Integer, String, String>() { - @Override - public Iterator<String> call(Integer key, Iterator<String> values) { + (FlatMapGroupsFunction<Integer, String, String>) (key, values) -> { StringBuilder sb = new StringBuilder(key.toString()); while (values.hasNext()) { sb.append(values.next()); } return Collections.singletonList(sb.toString()).iterator(); - } - }, + }, Encoders.STRING()); Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped.collectAsList())); Dataset<String> mapped2 = grouped.mapGroupsWithState( - new MapGroupsWithStateFunction<Integer, String, Long, String>() { - @Override - public String call(Integer key, Iterator<String> values, KeyedState<Long> s) { + (MapGroupsWithStateFunction<Integer, String, Long, String>) (key, values, s) -> { StringBuilder sb = new StringBuilder(key.toString()); while (values.hasNext()) { sb.append(values.next()); } return sb.toString(); - } }, Encoders.LONG(), Encoders.STRING()); @@ -242,27 +197,19 @@ public class JavaDatasetSuite implements Serializable { Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped2.collectAsList())); Dataset<String> flatMapped2 = grouped.flatMapGroupsWithState( - new FlatMapGroupsWithStateFunction<Integer, String, Long, String>() { - @Override - public Iterator<String> call(Integer key, Iterator<String> values, KeyedState<Long> s) { + (FlatMapGroupsWithStateFunction<Integer, String, Long, String>) (key, values, s) -> { StringBuilder sb = new StringBuilder(key.toString()); while (values.hasNext()) { sb.append(values.next()); } return Collections.singletonList(sb.toString()).iterator(); - } - }, + }, Encoders.LONG(), Encoders.STRING()); Assert.assertEquals(asSet("1a", "3foobar"), toSet(flatMapped2.collectAsList())); - Dataset<Tuple2<Integer, String>> reduced = grouped.reduceGroups(new ReduceFunction<String>() { - @Override - public String call(String v1, String v2) throws Exception { - return v1 + v2; - } - }); + Dataset<Tuple2<Integer, String>> reduced = grouped.reduceGroups((ReduceFunction<String>) (v1, v2) -> v1 + v2); Assert.assertEquals( asSet(tuple2(1, "a"), tuple2(3, "foobar")), @@ -271,29 +218,21 @@ public class JavaDatasetSuite implements Serializable { List<Integer> data2 = Arrays.asList(2, 6, 10); Dataset<Integer> ds2 = spark.createDataset(data2, Encoders.INT()); KeyValueGroupedDataset<Integer, Integer> grouped2 = ds2.groupByKey( - new MapFunction<Integer, Integer>() { - @Override - public Integer call(Integer v) throws Exception { - return v / 2; - } - }, + (MapFunction<Integer, Integer>) v -> v / 2, Encoders.INT()); Dataset<String> cogrouped = grouped.cogroup( grouped2, - new CoGroupFunction<Integer, String, Integer, String>() { - @Override - public Iterator<String> call(Integer key, Iterator<String> left, Iterator<Integer> right) { - StringBuilder sb = new StringBuilder(key.toString()); - while (left.hasNext()) { - sb.append(left.next()); - } - sb.append("#"); - while (right.hasNext()) { - sb.append(right.next()); - } - return Collections.singletonList(sb.toString()).iterator(); + (CoGroupFunction<Integer, String, Integer, String>) (key, left, right) -> { + StringBuilder sb = new StringBuilder(key.toString()); + while (left.hasNext()) { + sb.append(left.next()); + } + sb.append("#"); + while (right.hasNext()) { + sb.append(right.next()); } + return Collections.singletonList(sb.toString()).iterator(); }, Encoders.STRING()); @@ -703,11 +642,11 @@ public class JavaDatasetSuite implements Serializable { obj1.setD(new String[]{"hello", null}); obj1.setE(Arrays.asList("a", "b")); obj1.setF(Arrays.asList(100L, null, 200L)); - Map<Integer, String> map1 = new HashMap<Integer, String>(); + Map<Integer, String> map1 = new HashMap<>(); map1.put(1, "a"); map1.put(2, "b"); obj1.setG(map1); - Map<String, String> nestedMap1 = new HashMap<String, String>(); + Map<String, String> nestedMap1 = new HashMap<>(); nestedMap1.put("x", "1"); nestedMap1.put("y", "2"); Map<List<Long>, Map<String, String>> complexMap1 = new HashMap<>(); @@ -721,11 +660,11 @@ public class JavaDatasetSuite implements Serializable { obj2.setD(new String[]{null, "world"}); obj2.setE(Arrays.asList("x", "y")); obj2.setF(Arrays.asList(300L, null, 400L)); - Map<Integer, String> map2 = new HashMap<Integer, String>(); + Map<Integer, String> map2 = new HashMap<>(); map2.put(3, "c"); map2.put(4, "d"); obj2.setG(map2); - Map<String, String> nestedMap2 = new HashMap<String, String>(); + Map<String, String> nestedMap2 = new HashMap<>(); nestedMap2.put("q", "1"); nestedMap2.put("w", "2"); Map<List<Long>, Map<String, String>> complexMap2 = new HashMap<>(); @@ -1328,7 +1267,7 @@ public class JavaDatasetSuite implements Serializable { @Test public void test() { /* SPARK-15285 Large numbers of Nested JavaBeans generates more than 64KB java bytecode */ - List<NestedComplicatedJavaBean> data = new ArrayList<NestedComplicatedJavaBean>(); + List<NestedComplicatedJavaBean> data = new ArrayList<>(); data.add(NestedComplicatedJavaBean.newBuilder().build()); NestedComplicatedJavaBean obj3 = new NestedComplicatedJavaBean(); diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index bbaac5a339..250fa674d8 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -27,7 +27,6 @@ import org.junit.Test; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.api.java.UDF1; import org.apache.spark.sql.api.java.UDF2; import org.apache.spark.sql.types.DataTypes; @@ -54,16 +53,7 @@ public class JavaUDFSuite implements Serializable { @SuppressWarnings("unchecked") @Test public void udf1Test() { - // With Java 8 lambdas: - // sqlContext.registerFunction( - // "stringLengthTest", (String str) -> str.length(), DataType.IntegerType); - - spark.udf().register("stringLengthTest", new UDF1<String, Integer>() { - @Override - public Integer call(String str) { - return str.length(); - } - }, DataTypes.IntegerType); + spark.udf().register("stringLengthTest", (String str) -> str.length(), DataTypes.IntegerType); Row result = spark.sql("SELECT stringLengthTest('test')").head(); Assert.assertEquals(4, result.getInt(0)); @@ -72,18 +62,8 @@ public class JavaUDFSuite implements Serializable { @SuppressWarnings("unchecked") @Test public void udf2Test() { - // With Java 8 lambdas: - // sqlContext.registerFunction( - // "stringLengthTest", - // (String str1, String str2) -> str1.length() + str2.length, - // DataType.IntegerType); - - spark.udf().register("stringLengthTest", new UDF2<String, String, Integer>() { - @Override - public Integer call(String str1, String str2) { - return str1.length() + str2.length(); - } - }, DataTypes.IntegerType); + spark.udf().register("stringLengthTest", + (String str1, String str2) -> str1.length() + str2.length(), DataTypes.IntegerType); Row result = spark.sql("SELECT stringLengthTest('test', 'test2')").head(); Assert.assertEquals(9, result.getInt(0)); @@ -91,8 +71,8 @@ public class JavaUDFSuite implements Serializable { public static class StringLengthTest implements UDF2<String, String, Integer> { @Override - public Integer call(String str1, String str2) throws Exception { - return new Integer(str1.length() + str2.length()); + public Integer call(String str1, String str2) { + return str1.length() + str2.length(); } } @@ -113,12 +93,7 @@ public class JavaUDFSuite implements Serializable { @SuppressWarnings("unchecked") @Test public void udf4Test() { - spark.udf().register("inc", new UDF1<Long, Long>() { - @Override - public Long call(Long i) { - return i + 1; - } - }, DataTypes.LongType); + spark.udf().register("inc", (Long i) -> i + 1, DataTypes.LongType); spark.range(10).toDF("x").createOrReplaceTempView("tmp"); // This tests when Java UDFs are required to be the semantically same (See SPARK-9435). diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java index 9b7701003d..cb8ed83e5a 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java @@ -27,9 +27,6 @@ import java.util.Set; import scala.Tuple2; import com.google.common.collect.Sets; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.util.ManualClock; import org.junit.Assert; @@ -53,18 +50,14 @@ public class JavaMapWithStateSuite extends LocalJavaStreamingContext implements JavaPairDStream<String, Integer> wordsDstream = null; Function4<Time, String, Optional<Integer>, State<Boolean>, Optional<Double>> mappingFunc = - new Function4<Time, String, Optional<Integer>, State<Boolean>, Optional<Double>>() { - @Override - public Optional<Double> call( - Time time, String word, Optional<Integer> one, State<Boolean> state) { - // Use all State's methods here - state.exists(); - state.get(); - state.isTimingOut(); - state.remove(); - state.update(true); - return Optional.of(2.0); - } + (time, word, one, state) -> { + // Use all State's methods here + state.exists(); + state.get(); + state.isTimingOut(); + state.remove(); + state.update(true); + return Optional.of(2.0); }; JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream = @@ -78,17 +71,14 @@ public class JavaMapWithStateSuite extends LocalJavaStreamingContext implements stateDstream.stateSnapshots(); Function3<String, Optional<Integer>, State<Boolean>, Double> mappingFunc2 = - new Function3<String, Optional<Integer>, State<Boolean>, Double>() { - @Override - public Double call(String key, Optional<Integer> one, State<Boolean> state) { - // Use all State's methods here - state.exists(); - state.get(); - state.isTimingOut(); - state.remove(); - state.update(true); - return 2.0; - } + (key, one, state) -> { + // Use all State's methods here + state.exists(); + state.get(); + state.isTimingOut(); + state.remove(); + state.update(true); + return 2.0; }; JavaMapWithStateDStream<String, Integer, Boolean, Double> stateDstream2 = @@ -136,13 +126,10 @@ public class JavaMapWithStateSuite extends LocalJavaStreamingContext implements ); Function3<String, Optional<Integer>, State<Integer>, Integer> mappingFunc = - new Function3<String, Optional<Integer>, State<Integer>, Integer>() { - @Override - public Integer call(String key, Optional<Integer> value, State<Integer> state) { - int sum = value.orElse(0) + (state.exists() ? state.get() : 0); - state.update(sum); - return sum; - } + (key, value, state) -> { + int sum = value.orElse(0) + (state.exists() ? state.get() : 0); + state.update(sum); + return sum; }; testOperation( inputData, @@ -159,29 +146,15 @@ public class JavaMapWithStateSuite extends LocalJavaStreamingContext implements int numBatches = expectedOutputs.size(); JavaDStream<K> inputStream = JavaTestUtils.attachTestInputStream(ssc, input, 2); JavaMapWithStateDStream<K, Integer, S, T> mapWithStateDStream = - JavaPairDStream.fromJavaDStream(inputStream.map(new Function<K, Tuple2<K, Integer>>() { - @Override - public Tuple2<K, Integer> call(K x) { - return new Tuple2<>(x, 1); - } - })).mapWithState(mapWithStateSpec); - - final List<Set<T>> collectedOutputs = + JavaPairDStream.fromJavaDStream(inputStream.map(x -> new Tuple2<>(x, 1))).mapWithState(mapWithStateSpec); + + List<Set<T>> collectedOutputs = Collections.synchronizedList(new ArrayList<Set<T>>()); - mapWithStateDStream.foreachRDD(new VoidFunction<JavaRDD<T>>() { - @Override - public void call(JavaRDD<T> rdd) { - collectedOutputs.add(Sets.newHashSet(rdd.collect())); - } - }); - final List<Set<Tuple2<K, S>>> collectedStateSnapshots = + mapWithStateDStream.foreachRDD(rdd -> collectedOutputs.add(Sets.newHashSet(rdd.collect()))); + List<Set<Tuple2<K, S>>> collectedStateSnapshots = Collections.synchronizedList(new ArrayList<Set<Tuple2<K, S>>>()); - mapWithStateDStream.stateSnapshots().foreachRDD(new VoidFunction<JavaPairRDD<K, S>>() { - @Override - public void call(JavaPairRDD<K, S> rdd) { - collectedStateSnapshots.add(Sets.newHashSet(rdd.collect())); - } - }); + mapWithStateDStream.stateSnapshots().foreachRDD(rdd -> + collectedStateSnapshots.add(Sets.newHashSet(rdd.collect()))); BatchCounter batchCounter = new BatchCounter(ssc.ssc()); ssc.start(); ((ManualClock) ssc.ssc().scheduler().clock()) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java index 091ccbfd85..9156047244 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -58,24 +58,16 @@ public class JavaReceiverAPISuite implements Serializable { TestServer server = new TestServer(0); server.start(); - final AtomicLong dataCounter = new AtomicLong(0); + AtomicLong dataCounter = new AtomicLong(0); try { JavaStreamingContext ssc = new JavaStreamingContext("local[2]", "test", new Duration(200)); JavaReceiverInputDStream<String> input = ssc.receiverStream(new JavaSocketReceiver("localhost", server.port())); - JavaDStream<String> mapped = input.map(new Function<String, String>() { - @Override - public String call(String v1) { - return v1 + "."; - } - }); - mapped.foreachRDD(new VoidFunction<JavaRDD<String>>() { - @Override - public void call(JavaRDD<String> rdd) { - long count = rdd.count(); - dataCounter.addAndGet(count); - } + JavaDStream<String> mapped = input.map((Function<String, String>) v1 -> v1 + "."); + mapped.foreachRDD((VoidFunction<JavaRDD<String>>) rdd -> { + long count = rdd.count(); + dataCounter.addAndGet(count); }); ssc.start(); @@ -110,11 +102,7 @@ public class JavaReceiverAPISuite implements Serializable { @Override public void onStart() { - new Thread() { - @Override public void run() { - receive(); - } - }.start(); + new Thread(this::receive).start(); } @Override diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java index f02fa87f61..3f4e6ddb21 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaWriteAheadLogSuite.java @@ -23,7 +23,6 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; -import com.google.common.base.Function; import com.google.common.collect.Iterators; import org.apache.spark.SparkConf; import org.apache.spark.network.util.JavaUtils; @@ -81,12 +80,7 @@ public class JavaWriteAheadLogSuite extends WriteAheadLog { @Override public Iterator<ByteBuffer> readAll() { - return Iterators.transform(records.iterator(), new Function<Record,ByteBuffer>() { - @Override - public ByteBuffer apply(Record input) { - return input.buffer; - } - }); + return Iterators.transform(records.iterator(), input -> input.buffer); } @Override @@ -114,7 +108,7 @@ public class JavaWriteAheadLogSuite extends WriteAheadLog { String data1 = "data1"; WriteAheadLogRecordHandle handle = wal.write(JavaUtils.stringToBytes(data1), 1234); Assert.assertTrue(handle instanceof JavaWriteAheadLogSuiteHandle); - Assert.assertEquals(JavaUtils.bytesToString(wal.read(handle)), data1); + Assert.assertEquals(data1, JavaUtils.bytesToString(wal.read(handle))); wal.write(JavaUtils.stringToBytes("data2"), 1235); wal.write(JavaUtils.stringToBytes("data3"), 1236); diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java index 646cb97066..9948a4074c 100644 --- a/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java +++ b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java @@ -28,7 +28,6 @@ import org.apache.spark.streaming.StateSpec; import org.apache.spark.streaming.Time; import scala.Tuple2; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; import org.junit.Assert; import org.junit.Test; @@ -101,7 +100,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ while (in.hasNext()) { out = out + in.next().toUpperCase(); } - return Lists.newArrayList(out).iterator(); + return Arrays.asList(out).iterator(); }); JavaTestUtils.attachTestOutputStream(mapped); List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -240,7 +239,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ JavaTestUtils.attachTestOutputStream(joined); List<List<Tuple2<String, Tuple2<String, String>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - List<Set<Tuple2<String, Tuple2<String, String>>>> unorderedResult = Lists.newArrayList(); + List<Set<Tuple2<String, Tuple2<String, String>>>> unorderedResult = new ArrayList<>(); for (List<Tuple2<String, Tuple2<String, String>>> res : result) { unorderedResult.add(Sets.newHashSet(res)); } @@ -315,7 +314,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ JavaPairDStream<Integer, String> pairStream1 = JavaPairDStream.fromJavaDStream( JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1)); - List<JavaDStream<?>> listOfDStreams1 = Arrays.<JavaDStream<?>>asList(stream1, stream2); + List<JavaDStream<?>> listOfDStreams1 = Arrays.asList(stream1, stream2); // This is just to test whether this transform to JavaStream compiles JavaDStream<Long> transformed1 = ssc.transform( @@ -325,7 +324,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ }); List<JavaDStream<?>> listOfDStreams2 = - Arrays.<JavaDStream<?>>asList(stream1, stream2, pairStream1.toJavaDStream()); + Arrays.asList(stream1, stream2, pairStream1.toJavaDStream()); JavaPairDStream<Integer, Tuple2<Integer, String>> transformed2 = ssc.transformToPair( listOfDStreams2, (List<JavaRDD<?>> listOfRDDs, Time time) -> { @@ -358,7 +357,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaDStream<String> flatMapped = stream.flatMap( - s -> Lists.newArrayList(s.split("(?!^)")).iterator()); + s -> Arrays.asList(s.split("(?!^)")).iterator()); JavaTestUtils.attachTestOutputStream(flatMapped); List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -401,7 +400,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<Integer, String> flatMapped = stream.flatMapToPair(s -> { - List<Tuple2<Integer, String>> out = Lists.newArrayList(); + List<Tuple2<Integer, String>> out = new ArrayList<>(); for (String letter : s.split("(?!^)")) { out.add(new Tuple2<>(s.length(), letter)); } @@ -420,7 +419,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ */ public static <T extends Comparable<T>> void assertOrderInvariantEquals( List<List<T>> expected, List<List<T>> actual) { - expected.forEach(list -> Collections.sort(list)); + expected.forEach(Collections::sort); List<List<T>> sortedActual = new ArrayList<>(); actual.forEach(list -> { List<T> sortedList = new ArrayList<>(list); @@ -491,7 +490,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<Integer, String> reversed = pairStream.mapToPair(x -> x.swap()); + JavaPairDStream<Integer, String> reversed = pairStream.mapToPair(Tuple2::swap); JavaTestUtils.attachTestOutputStream(reversed); List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -543,7 +542,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaDStream<Integer> reversed = pairStream.map(in -> in._2()); + JavaDStream<Integer> reversed = pairStream.map(Tuple2::_2); JavaTestUtils.attachTestOutputStream(reversed); List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -629,7 +628,7 @@ public class Java8APISuite extends LocalJavaStreamingContext implements Serializ ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<String, Integer> combined = pairStream.<Integer>combineByKey(i -> i, + JavaPairDStream<String, Integer> combined = pairStream.combineByKey(i -> i, (x, y) -> x + y, (x, y) -> x + y, new HashPartitioner(2)); JavaTestUtils.attachTestOutputStream(combined); diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java index 8d24104d78..b966cbdca0 100644 --- a/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java @@ -33,7 +33,6 @@ import org.apache.spark.streaming.Time; import scala.Tuple2; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; @@ -123,12 +122,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(9,4)); JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() { - @Override - public Integer call(String s) { - return s.length(); - } - }); + JavaDStream<Integer> letterCount = stream.map(String::length); JavaTestUtils.attachTestOutputStream(letterCount); List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -194,12 +188,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList("yankees")); JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream<String> filtered = stream.filter(new Function<String, Boolean>() { - @Override - public Boolean call(String s) { - return s.contains("a"); - } - }); + JavaDStream<String> filtered = stream.filter(s -> s.contains("a")); JavaTestUtils.attachTestOutputStream(filtered); List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -276,17 +265,13 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList("YANKEESRED SOX")); JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream<String> mapped = stream.mapPartitions( - new FlatMapFunction<Iterator<String>, String>() { - @Override - public Iterator<String> call(Iterator<String> in) { - StringBuilder out = new StringBuilder(); - while (in.hasNext()) { - out.append(in.next().toUpperCase(Locale.ENGLISH)); - } - return Arrays.asList(out.toString()).iterator(); - } - }); + JavaDStream<String> mapped = stream.mapPartitions(in -> { + StringBuilder out = new StringBuilder(); + while (in.hasNext()) { + out.append(in.next().toUpperCase(Locale.ENGLISH)); + } + return Arrays.asList(out.toString()).iterator(); + }); JavaTestUtils.attachTestOutputStream(mapped); List<List<String>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -416,18 +401,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(9,10,11)); JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream<Integer> transformed = stream.transform( - new Function<JavaRDD<Integer>, JavaRDD<Integer>>() { - @Override - public JavaRDD<Integer> call(JavaRDD<Integer> in) { - return in.map(new Function<Integer, Integer>() { - @Override - public Integer call(Integer i) { - return i + 2; - } - }); - } - }); + JavaDStream<Integer> transformed = stream.transform(in -> in.map(i -> i + 2)); JavaTestUtils.attachTestOutputStream(transformed); List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -448,71 +422,21 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream( JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1)); - stream.transform( - new Function<JavaRDD<Integer>, JavaRDD<Integer>>() { - @Override - public JavaRDD<Integer> call(JavaRDD<Integer> in) { - return null; - } - } - ); + stream.transform(in -> null); - stream.transform( - new Function2<JavaRDD<Integer>, Time, JavaRDD<Integer>>() { - @Override public JavaRDD<Integer> call(JavaRDD<Integer> in, Time time) { - return null; - } - } - ); + stream.transform((in, time) -> null); - stream.transformToPair( - new Function<JavaRDD<Integer>, JavaPairRDD<String, Integer>>() { - @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in) { - return null; - } - } - ); + stream.transformToPair(in -> null); - stream.transformToPair( - new Function2<JavaRDD<Integer>, Time, JavaPairRDD<String, Integer>>() { - @Override public JavaPairRDD<String, Integer> call(JavaRDD<Integer> in, Time time) { - return null; - } - } - ); + stream.transformToPair((in, time) -> null); - pairStream.transform( - new Function<JavaPairRDD<String, Integer>, JavaRDD<Integer>>() { - @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in) { - return null; - } - } - ); + pairStream.transform(in -> null); - pairStream.transform( - new Function2<JavaPairRDD<String, Integer>, Time, JavaRDD<Integer>>() { - @Override public JavaRDD<Integer> call(JavaPairRDD<String, Integer> in, Time time) { - return null; - } - } - ); + pairStream.transform((in, time) -> null); - pairStream.transformToPair( - new Function<JavaPairRDD<String, Integer>, JavaPairRDD<String, String>>() { - @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in) { - return null; - } - } - ); + pairStream.transformToPair(in -> null); - pairStream.transformToPair( - new Function2<JavaPairRDD<String, Integer>, Time, JavaPairRDD<String, String>>() { - @Override public JavaPairRDD<String, String> call(JavaPairRDD<String, Integer> in, - Time time) { - return null; - } - } - ); + pairStream.transformToPair((in, time) -> null); } @@ -558,19 +482,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaPairDStream<String, Tuple2<String, String>> joined = pairStream1.transformWithToPair( pairStream2, - new Function3< - JavaPairRDD<String, String>, - JavaPairRDD<String, String>, - Time, - JavaPairRDD<String, Tuple2<String, String>>>() { - @Override - public JavaPairRDD<String, Tuple2<String, String>> call( - JavaPairRDD<String, String> rdd1, - JavaPairRDD<String, String> rdd2, - Time time) { - return rdd1.join(rdd2); - } - } + (rdd1, rdd2, time) -> rdd1.join(rdd2) ); JavaTestUtils.attachTestOutputStream(joined); @@ -603,100 +515,21 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaPairDStream<Double, Character> pairStream2 = JavaPairDStream.fromJavaDStream( JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1)); - stream1.transformWith( - stream2, - new Function3<JavaRDD<Integer>, JavaRDD<String>, Time, JavaRDD<Double>>() { - @Override - public JavaRDD<Double> call(JavaRDD<Integer> rdd1, JavaRDD<String> rdd2, Time time) { - return null; - } - } - ); + stream1.transformWith(stream2, (rdd1, rdd2, time) -> null); - stream1.transformWith( - 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) { - return null; - } - } - ); + stream1.transformWith(pairStream1, (rdd1, rdd2, time) -> null); - stream1.transformWithToPair( - 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) { - return null; - } - } - ); + stream1.transformWithToPair(stream2, (rdd1, rdd2, time) -> null); - stream1.transformWithToPair( - pairStream1, - 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) { - return null; - } - } - ); + stream1.transformWithToPair(pairStream1, (rdd1, rdd2, time) -> null); - pairStream1.transformWith( - 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) { - return null; - } - } - ); + pairStream1.transformWith(stream2, (rdd1, rdd2, time) -> null); - pairStream1.transformWith( - pairStream1, - 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) { - return null; - } - } - ); + pairStream1.transformWith(pairStream1, (rdd1, rdd2, time) -> null); - pairStream1.transformWithToPair( - stream2, - 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) { - return null; - } - } - ); + pairStream1.transformWithToPair(stream2, (rdd1, rdd2, time) -> null); - pairStream1.transformWithToPair( - pairStream2, - 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) { - return null; - } - } - ); + pairStream1.transformWithToPair(pairStream2, (rdd1, rdd2, time) -> null); } @SuppressWarnings("unchecked") @@ -727,44 +560,32 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaPairDStream<Integer, String> pairStream1 = JavaPairDStream.fromJavaDStream( JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1)); - List<JavaDStream<?>> listOfDStreams1 = Arrays.<JavaDStream<?>>asList(stream1, stream2); + List<JavaDStream<?>> listOfDStreams1 = Arrays.asList(stream1, stream2); // This is just to test whether this transform to JavaStream compiles ssc.transform( listOfDStreams1, - new Function2<List<JavaRDD<?>>, Time, JavaRDD<Long>>() { - @Override - public JavaRDD<Long> call(List<JavaRDD<?>> listOfRDDs, Time time) { - Assert.assertEquals(2, listOfRDDs.size()); - return null; - } + (listOfRDDs, time) -> { + Assert.assertEquals(2, listOfRDDs.size()); + return null; } ); List<JavaDStream<?>> listOfDStreams2 = - Arrays.<JavaDStream<?>>asList(stream1, stream2, pairStream1.toJavaDStream()); + Arrays.asList(stream1, stream2, pairStream1.toJavaDStream()); JavaPairDStream<Integer, Tuple2<Integer, String>> transformed2 = ssc.transformToPair( listOfDStreams2, - new Function2<List<JavaRDD<?>>, Time, JavaPairRDD<Integer, Tuple2<Integer, String>>>() { - @Override - 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); - JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3); - PairFunction<Integer, Integer, Integer> mapToTuple = - new PairFunction<Integer, Integer, Integer>() { - @Override - public Tuple2<Integer, Integer> call(Integer i) { - return new Tuple2<>(i, i); - } - }; - return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); - } + (listOfRDDs, 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); + JavaPairRDD<Integer, String> prdd3 = JavaPairRDD.fromJavaRDD(rdd3); + PairFunction<Integer, Integer, Integer> mapToTuple = + (PairFunction<Integer, Integer, Integer>) i -> new Tuple2<>(i, i); + return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); } ); JavaTestUtils.attachTestOutputStream(transformed2); @@ -787,12 +608,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList("a","t","h","l","e","t","i","c","s")); JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream<String> flatMapped = stream.flatMap(new FlatMapFunction<String, String>() { - @Override - public Iterator<String> call(String x) { - return Arrays.asList(x.split("(?!^)")).iterator(); - } - }); + JavaDStream<String> flatMapped = stream.flatMap(x -> Arrays.asList(x.split("(?!^)")).iterator()); JavaTestUtils.attachTestOutputStream(flatMapped); List<List<String>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -811,25 +627,13 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaDStream<Integer> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaTestUtils.attachTestOutputStream(stream.count()); // dummy output - stream.foreachRDD(new VoidFunction<JavaRDD<Integer>>() { - @Override - public void call(JavaRDD<Integer> rdd) { - accumRdd.add(1); - rdd.foreach(new VoidFunction<Integer>() { - @Override - public void call(Integer i) { - accumEle.add(1); - } - }); - } + stream.foreachRDD(rdd -> { + accumRdd.add(1); + rdd.foreach(i -> accumEle.add(1)); }); // This is a test to make sure foreachRDD(VoidFunction2) can be called from Java - stream.foreachRDD(new VoidFunction2<JavaRDD<Integer>, Time>() { - @Override - public void call(JavaRDD<Integer> rdd, Time time) { - } - }); + stream.foreachRDD((rdd, time) -> {}); JavaTestUtils.runStreams(ssc, 2, 2); @@ -873,16 +677,12 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa new Tuple2<>(9, "s"))); JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream<Integer, String> flatMapped = stream.flatMapToPair( - new PairFlatMapFunction<String, Integer, String>() { - @Override - public Iterator<Tuple2<Integer, String>> call(String in) { - List<Tuple2<Integer, String>> out = new ArrayList<>(); - for (String letter: in.split("(?!^)")) { - out.add(new Tuple2<>(in.length(), letter)); - } - return out.iterator(); + JavaPairDStream<Integer, String> flatMapped = stream.flatMapToPair(in -> { + List<Tuple2<Integer, String>> out = new ArrayList<>(); + for (String letter : in.split("(?!^)")) { + out.add(new Tuple2<>(in.length(), letter)); } + return out.iterator(); }); JavaTestUtils.attachTestOutputStream(flatMapped); List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -949,21 +749,10 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa Arrays.asList(new Tuple2<>("yankees", 7))); JavaDStream<String> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream<String, Integer> pairStream = stream.mapToPair( - new PairFunction<String, String, Integer>() { - @Override - public Tuple2<String, Integer> call(String in) { - return new Tuple2<>(in, in.length()); - } - }); + JavaPairDStream<String, Integer> pairStream = + stream.mapToPair(in -> new Tuple2<>(in, in.length())); - JavaPairDStream<String, Integer> filtered = pairStream.filter( - new Function<Tuple2<String, Integer>, Boolean>() { - @Override - public Boolean call(Tuple2<String, Integer> in) { - return in._1().contains("a"); - } - }); + JavaPairDStream<String, Integer> filtered = pairStream.filter(in -> in._1().contains("a")); JavaTestUtils.attachTestOutputStream(filtered); List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1014,13 +803,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa 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>() { - @Override - public Tuple2<Integer, String> call(Tuple2<String, Integer> in) { - return in.swap(); - } - }); + JavaPairDStream<Integer, String> reversed = pairStream.mapToPair(Tuple2::swap); JavaTestUtils.attachTestOutputStream(reversed); List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1048,18 +831,14 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa 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>() { - @Override - public Iterator<Tuple2<Integer, String>> call(Iterator<Tuple2<String, Integer>> in) { - List<Tuple2<Integer, String>> out = new LinkedList<>(); - while (in.hasNext()) { - Tuple2<String, Integer> next = in.next(); - out.add(next.swap()); - } - return out.iterator(); - } - }); + JavaPairDStream<Integer, String> reversed = pairStream.mapPartitionsToPair(in -> { + List<Tuple2<Integer, String>> out = new LinkedList<>(); + while (in.hasNext()) { + Tuple2<String, Integer> next = in.next(); + out.add(next.swap()); + } + return out.iterator(); + }); JavaTestUtils.attachTestOutputStream(reversed); List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1079,13 +858,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa 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>() { - @Override - public Integer call(Tuple2<String, Integer> in) { - return in._2(); - } - }); + JavaDStream<Integer> reversed = pairStream.map(in -> in._2()); JavaTestUtils.attachTestOutputStream(reversed); List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1119,17 +892,13 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaDStream<Tuple2<String, Integer>> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<Integer, String> flatMapped = pairStream.flatMapToPair( - new PairFlatMapFunction<Tuple2<String, Integer>, Integer, String>() { - @Override - public Iterator<Tuple2<Integer, String>> call(Tuple2<String, Integer> in) { - List<Tuple2<Integer, String>> out = new LinkedList<>(); - for (Character s : in._1().toCharArray()) { - out.add(new Tuple2<>(in._2(), s.toString())); - } - return out.iterator(); - } - }); + JavaPairDStream<Integer, String> flatMapped = pairStream.flatMapToPair(in -> { + List<Tuple2<Integer, String>> out = new LinkedList<>(); + for (Character s : in._1().toCharArray()) { + out.add(new Tuple2<>(in._2(), s.toString())); + } + return out.iterator(); + }); JavaTestUtils.attachTestOutputStream(flatMapped); List<List<Tuple2<Integer, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1216,12 +985,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream<String, Integer> combined = pairStream.combineByKey( - new Function<Integer, Integer>() { - @Override - public Integer call(Integer i) { - return i; - } - }, new IntegerSum(), new IntegerSum(), new HashPartitioner(2)); + i -> i, new IntegerSum(), new IntegerSum(), new HashPartitioner(2)); JavaTestUtils.attachTestOutputStream(combined); List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1345,20 +1109,16 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey( - new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() { - @Override - public Optional<Integer> call(List<Integer> values, Optional<Integer> state) { - int out = 0; - if (state.isPresent()) { - out += state.get(); - } - for (Integer v : values) { - out += v; - } - return Optional.of(out); - } - }); + JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey((values, state) -> { + int out = 0; + if (state.isPresent()) { + out += state.get(); + } + for (Integer v : values) { + out += v; + } + return Optional.of(out); + }); JavaTestUtils.attachTestOutputStream(updated); List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1389,20 +1149,16 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream<String, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey( - new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() { - @Override - public Optional<Integer> call(List<Integer> values, Optional<Integer> state) { - int out = 0; - if (state.isPresent()) { - out += state.get(); - } - for (Integer v : values) { - out += v; - } - return Optional.of(out); - } - }, new HashPartitioner(1), initialRDD); + JavaPairDStream<String, Integer> updated = pairStream.updateStateByKey((values, state) -> { + int out = 0; + if (state.isPresent()) { + out += state.get(); + } + for (Integer v : values) { + out += v; + } + return Optional.of(out); + }, new HashPartitioner(1), initialRDD); JavaTestUtils.attachTestOutputStream(updated); List<List<Tuple2<String, Integer>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1500,13 +1256,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ssc, inputData, 1); JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<Integer, Integer> sorted = pairStream.transformToPair( - new Function<JavaPairRDD<Integer, Integer>, JavaPairRDD<Integer, Integer>>() { - @Override - public JavaPairRDD<Integer, Integer> call(JavaPairRDD<Integer, Integer> in) { - return in.sortByKey(); - } - }); + JavaPairDStream<Integer, Integer> sorted = pairStream.transformToPair(in -> in.sortByKey()); JavaTestUtils.attachTestOutputStream(sorted); List<List<Tuple2<Integer, Integer>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1537,18 +1287,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ssc, inputData, 1); JavaPairDStream<Integer, Integer> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaDStream<Integer> firstParts = pairStream.transform( - new Function<JavaPairRDD<Integer, Integer>, JavaRDD<Integer>>() { - @Override - public JavaRDD<Integer> call(JavaPairRDD<Integer, Integer> in) { - return in.map(new Function<Tuple2<Integer, Integer>, Integer>() { - @Override - public Integer call(Tuple2<Integer, Integer> in2) { - return in2._1(); - } - }); - } - }); + JavaDStream<Integer> firstParts = pairStream.transform(in -> in.map(in2 -> in2._1())); JavaTestUtils.attachTestOutputStream(firstParts); List<List<Integer>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1575,12 +1314,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ssc, inputData, 1); JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<String, String> mapped = pairStream.mapValues(new Function<String, String>() { - @Override - public String call(String s) { - return s.toUpperCase(Locale.ENGLISH); - } - }); + JavaPairDStream<String, String> mapped = pairStream.mapValues(s -> s.toUpperCase(Locale.ENGLISH)); JavaTestUtils.attachTestOutputStream(mapped); List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1616,16 +1350,12 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa JavaPairDStream<String, String> pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream<String, String> flatMapped = pairStream.flatMapValues( - new Function<String, Iterable<String>>() { - @Override - public Iterable<String> call(String in) { - List<String> out = new ArrayList<>(); - out.add(in + "1"); - out.add(in + "2"); - return out; - } - }); + JavaPairDStream<String, String> flatMapped = pairStream.flatMapValues(in -> { + List<String> out = new ArrayList<>(); + out.add(in + "1"); + out.add(in + "2"); + return out; + }); JavaTestUtils.attachTestOutputStream(flatMapped); List<List<Tuple2<String, String>>> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -1795,12 +1525,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ssc.checkpoint(tempDir.getAbsolutePath()); JavaDStream<String> stream = JavaCheckpointTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream<Integer> letterCount = stream.map(new Function<String, Integer>() { - @Override - public Integer call(String s) { - return s.length(); - } - }); + JavaDStream<Integer> letterCount = stream.map(String::length); JavaCheckpointTestUtils.attachTestOutputStream(letterCount); List<List<Integer>> initialResult = JavaTestUtils.runStreams(ssc, 1, 1); @@ -1822,7 +1547,7 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa public void testContextGetOrCreate() throws InterruptedException { ssc.stop(); - final SparkConf conf = new SparkConf() + SparkConf conf = new SparkConf() .setMaster("local[2]") .setAppName("test") .set("newContext", "true"); @@ -1835,13 +1560,10 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa // Function to create JavaStreamingContext without any output operations // (used to detect the new context) - final AtomicBoolean newContextCreated = new AtomicBoolean(false); - Function0<JavaStreamingContext> creatingFunc = new Function0<JavaStreamingContext>() { - @Override - public JavaStreamingContext call() { - newContextCreated.set(true); - return new JavaStreamingContext(conf, Seconds.apply(1)); - } + AtomicBoolean newContextCreated = new AtomicBoolean(false); + Function0<JavaStreamingContext> creatingFunc = () -> { + newContextCreated.set(true); + return new JavaStreamingContext(conf, Seconds.apply(1)); }; newContextCreated.set(false); @@ -1912,18 +1634,15 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa ssc.socketStream( "localhost", 12345, - new Function<InputStream, Iterable<String>>() { - @Override - public Iterable<String> call(InputStream in) throws IOException { - List<String> out = new ArrayList<>(); - try (BufferedReader reader = new BufferedReader( - new InputStreamReader(in, StandardCharsets.UTF_8))) { - for (String line; (line = reader.readLine()) != null;) { - out.add(line); - } + in -> { + List<String> out = new ArrayList<>(); + try (BufferedReader reader = new BufferedReader( + new InputStreamReader(in, StandardCharsets.UTF_8))) { + for (String line; (line = reader.readLine()) != null;) { + out.add(line); } - return out; } + return out; }, StorageLevel.MEMORY_ONLY()); } @@ -1952,21 +1671,10 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa LongWritable.class, Text.class, TextInputFormat.class, - new Function<Path, Boolean>() { - @Override - public Boolean call(Path v1) { - return Boolean.TRUE; - } - }, + v1 -> Boolean.TRUE, true); - JavaDStream<String> test = inputStream.map( - new Function<Tuple2<LongWritable, Text>, String>() { - @Override - public String call(Tuple2<LongWritable, Text> v1) { - return v1._2().toString(); - } - }); + JavaDStream<String> test = inputStream.map(v1 -> v1._2().toString()); JavaTestUtils.attachTestOutputStream(test); List<List<String>> result = JavaTestUtils.runStreams(ssc, 1, 1); -- GitLab