From 76d74090d60f74412bd45487e8db6aff2e8343a2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 4 Aug 2015 12:02:26 +0100 Subject: [SPARK-9534] [BUILD] Enable javac lint for scalac parity; fix a lot of build warnings, 1.5.0 edition Enable most javac lint warnings; fix a lot of build warnings. In a few cases, touch up surrounding code in the process. I'll explain several of the changes inline in comments. Author: Sean Owen Closes #7862 from srowen/SPARK-9534 and squashes the following commits: ea51618 [Sean Owen] Enable most javac lint warnings; fix a lot of build warnings. In a few cases, touch up surrounding code in the process. --- .../spark/network/protocol/ChunkFetchFailure.java | 5 +++ .../spark/network/protocol/ChunkFetchRequest.java | 5 +++ .../spark/network/protocol/ChunkFetchSuccess.java | 5 +++ .../apache/spark/network/protocol/RpcFailure.java | 5 +++ .../apache/spark/network/protocol/RpcRequest.java | 5 +++ .../apache/spark/network/protocol/RpcResponse.java | 5 +++ .../apache/spark/network/TestManagedBuffer.java | 5 +++ .../apache/spark/network/sasl/SparkSaslSuite.java | 16 ++++---- .../shuffle/ExternalShuffleBlockHandlerSuite.java | 6 ++- .../network/shuffle/RetryingBlockFetcherSuite.java | 47 ++++++++++++---------- 10 files changed, 73 insertions(+), 31 deletions(-) (limited to 'network') diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java index f76bb49e87..f0363830b6 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -52,6 +52,11 @@ public final class ChunkFetchFailure implements ResponseMessage { return new ChunkFetchFailure(streamChunkId, errorString); } + @Override + public int hashCode() { + return Objects.hashCode(streamChunkId, errorString); + } + @Override public boolean equals(Object other) { if (other instanceof ChunkFetchFailure) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java index 980947cf13..5a173af54f 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java @@ -48,6 +48,11 @@ public final class ChunkFetchRequest implements RequestMessage { return new ChunkFetchRequest(StreamChunkId.decode(buf)); } + @Override + public int hashCode() { + return streamChunkId.hashCode(); + } + @Override public boolean equals(Object other) { if (other instanceof ChunkFetchRequest) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java index ff4936470c..c962fb7ecf 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java @@ -61,6 +61,11 @@ public final class ChunkFetchSuccess implements ResponseMessage { return new ChunkFetchSuccess(streamChunkId, managedBuf); } + @Override + public int hashCode() { + return Objects.hashCode(streamChunkId, buffer); + } + @Override public boolean equals(Object other) { if (other instanceof ChunkFetchSuccess) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java index 6b991375fc..2dfc7876ba 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -50,6 +50,11 @@ public final class RpcFailure implements ResponseMessage { return new RpcFailure(requestId, errorString); } + @Override + public int hashCode() { + return Objects.hashCode(requestId, errorString); + } + @Override public boolean equals(Object other) { if (other instanceof RpcFailure) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java index cdee0b0e03..745039db74 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -59,6 +59,11 @@ public final class RpcRequest implements RequestMessage { return new RpcRequest(requestId, message); } + @Override + public int hashCode() { + return Objects.hashCode(requestId, Arrays.hashCode(message)); + } + @Override public boolean equals(Object other) { if (other instanceof RpcRequest) { diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java index 0a62e09a81..1671cd444f 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -50,6 +50,11 @@ public final class RpcResponse implements ResponseMessage { return new RpcResponse(requestId, response); } + @Override + public int hashCode() { + return Objects.hashCode(requestId, Arrays.hashCode(response)); + } + @Override public boolean equals(Object other) { if (other instanceof RpcResponse) { diff --git a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java index 38113a918f..83c90f9eff 100644 --- a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java +++ b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java @@ -80,6 +80,11 @@ public class TestManagedBuffer extends ManagedBuffer { return underlying.convertToNetty(); } + @Override + public int hashCode() { + return underlying.hashCode(); + } + @Override public boolean equals(Object other) { if (other instanceof ManagedBuffer) { diff --git a/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index be6632bb8c..8104004847 100644 --- a/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -17,11 +17,11 @@ package org.apache.spark.network.sasl; -import static com.google.common.base.Charsets.UTF_8; import static org.junit.Assert.*; import static org.mockito.Mockito.*; import java.io.File; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Random; @@ -138,8 +138,8 @@ public class SparkSaslSuite { public Void answer(InvocationOnMock invocation) { byte[] message = (byte[]) invocation.getArguments()[1]; RpcResponseCallback cb = (RpcResponseCallback) invocation.getArguments()[2]; - assertEquals("Ping", new String(message, UTF_8)); - cb.onSuccess("Pong".getBytes(UTF_8)); + assertEquals("Ping", new String(message, StandardCharsets.UTF_8)); + cb.onSuccess("Pong".getBytes(StandardCharsets.UTF_8)); return null; } }) @@ -148,8 +148,9 @@ public class SparkSaslSuite { SaslTestCtx ctx = new SaslTestCtx(rpcHandler, encrypt, false); try { - byte[] response = ctx.client.sendRpcSync("Ping".getBytes(UTF_8), TimeUnit.SECONDS.toMillis(10)); - assertEquals("Pong", new String(response, UTF_8)); + byte[] response = ctx.client.sendRpcSync("Ping".getBytes(StandardCharsets.UTF_8), + TimeUnit.SECONDS.toMillis(10)); + assertEquals("Pong", new String(response, StandardCharsets.UTF_8)); } finally { ctx.close(); } @@ -235,7 +236,7 @@ public class SparkSaslSuite { final String blockSizeConf = "spark.network.sasl.maxEncryptedBlockSize"; System.setProperty(blockSizeConf, "1k"); - final AtomicReference response = new AtomicReference(); + final AtomicReference response = new AtomicReference<>(); final File file = File.createTempFile("sasltest", ".txt"); SaslTestCtx ctx = null; try { @@ -321,7 +322,8 @@ public class SparkSaslSuite { SaslTestCtx ctx = null; try { ctx = new SaslTestCtx(mock(RpcHandler.class), true, true); - ctx.client.sendRpcSync("Ping".getBytes(UTF_8), TimeUnit.SECONDS.toMillis(10)); + ctx.client.sendRpcSync("Ping".getBytes(StandardCharsets.UTF_8), + TimeUnit.SECONDS.toMillis(10)); fail("Should have failed to send RPC to server."); } catch (Exception e) { assertFalse(e.getCause() instanceof TimeoutException); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 73374cdc77..1d197497b7 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -90,9 +90,11 @@ public class ExternalShuffleBlockHandlerSuite { (StreamHandle) BlockTransferMessage.Decoder.fromByteArray(response.getValue()); assertEquals(2, handle.numChunks); - ArgumentCaptor stream = ArgumentCaptor.forClass(Iterator.class); + @SuppressWarnings("unchecked") + ArgumentCaptor> stream = (ArgumentCaptor>) + (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); verify(streamManager, times(1)).registerStream(stream.capture()); - Iterator buffers = (Iterator) stream.getValue(); + Iterator buffers = stream.getValue(); assertEquals(block0Marker, buffers.next()); assertEquals(block1Marker, buffers.next()); assertFalse(buffers.hasNext()); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java index 1ad0d72ae5..06e46f9241 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -20,7 +20,9 @@ package org.apache.spark.network.shuffle; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.LinkedHashSet; +import java.util.List; import java.util.Map; import com.google.common.collect.ImmutableMap; @@ -67,13 +69,13 @@ public class RetryingBlockFetcherSuite { public void testNoFailures() throws IOException { BlockFetchingListener listener = mock(BlockFetchingListener.class); - Map[] interactions = new Map[] { + List> interactions = Arrays.asList( // Immediately return both blocks successfully. ImmutableMap.builder() .put("b0", block0) .put("b1", block1) - .build(), - }; + .build() + ); performInteractions(interactions, listener); @@ -86,13 +88,13 @@ public class RetryingBlockFetcherSuite { public void testUnrecoverableFailure() throws IOException { BlockFetchingListener listener = mock(BlockFetchingListener.class); - Map[] interactions = new Map[] { + List> interactions = Arrays.asList( // b0 throws a non-IOException error, so it will be failed without retry. ImmutableMap.builder() .put("b0", new RuntimeException("Ouch!")) .put("b1", block1) - .build(), - }; + .build() + ); performInteractions(interactions, listener); @@ -105,7 +107,7 @@ public class RetryingBlockFetcherSuite { public void testSingleIOExceptionOnFirst() throws IOException { BlockFetchingListener listener = mock(BlockFetchingListener.class); - Map[] interactions = new Map[] { + List> interactions = Arrays.asList( // IOException will cause a retry. Since b0 fails, we will retry both. ImmutableMap.builder() .put("b0", new IOException("Connection failed or something")) @@ -114,8 +116,8 @@ public class RetryingBlockFetcherSuite { ImmutableMap.builder() .put("b0", block0) .put("b1", block1) - .build(), - }; + .build() + ); performInteractions(interactions, listener); @@ -128,7 +130,7 @@ public class RetryingBlockFetcherSuite { public void testSingleIOExceptionOnSecond() throws IOException { BlockFetchingListener listener = mock(BlockFetchingListener.class); - Map[] interactions = new Map[] { + List> interactions = Arrays.asList( // IOException will cause a retry. Since b1 fails, we will not retry b0. ImmutableMap.builder() .put("b0", block0) @@ -136,8 +138,8 @@ public class RetryingBlockFetcherSuite { .build(), ImmutableMap.builder() .put("b1", block1) - .build(), - }; + .build() + ); performInteractions(interactions, listener); @@ -150,7 +152,7 @@ public class RetryingBlockFetcherSuite { public void testTwoIOExceptions() throws IOException { BlockFetchingListener listener = mock(BlockFetchingListener.class); - Map[] interactions = new Map[] { + List> interactions = Arrays.asList( // b0's IOException will trigger retry, b1's will be ignored. ImmutableMap.builder() .put("b0", new IOException()) @@ -164,8 +166,8 @@ public class RetryingBlockFetcherSuite { // b1 returns successfully within 2 retries. ImmutableMap.builder() .put("b1", block1) - .build(), - }; + .build() + ); performInteractions(interactions, listener); @@ -178,7 +180,7 @@ public class RetryingBlockFetcherSuite { public void testThreeIOExceptions() throws IOException { BlockFetchingListener listener = mock(BlockFetchingListener.class); - Map[] interactions = new Map[] { + List> interactions = Arrays.asList( // b0's IOException will trigger retry, b1's will be ignored. ImmutableMap.builder() .put("b0", new IOException()) @@ -196,8 +198,8 @@ public class RetryingBlockFetcherSuite { // This is not reached -- b1 has failed. ImmutableMap.builder() .put("b1", block1) - .build(), - }; + .build() + ); performInteractions(interactions, listener); @@ -210,7 +212,7 @@ public class RetryingBlockFetcherSuite { public void testRetryAndUnrecoverable() throws IOException { BlockFetchingListener listener = mock(BlockFetchingListener.class); - Map[] interactions = new Map[] { + List> interactions = Arrays.asList( // b0's IOException will trigger retry, subsequent messages will be ignored. ImmutableMap.builder() .put("b0", new IOException()) @@ -226,8 +228,8 @@ public class RetryingBlockFetcherSuite { // b2 succeeds in its last retry. ImmutableMap.builder() .put("b2", block2) - .build(), - }; + .build() + ); performInteractions(interactions, listener); @@ -248,7 +250,8 @@ public class RetryingBlockFetcherSuite { * subset of the original blocks in a second interaction. */ @SuppressWarnings("unchecked") - private void performInteractions(final Map[] interactions, BlockFetchingListener listener) + private static void performInteractions(List> interactions, + BlockFetchingListener listener) throws IOException { TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); -- cgit v1.2.3