Skip to content

Commit e97fc7f

Browse files
committed
[SPARK-13423][WIP][CORE][SQL][STREAMING] Static analysis fixes for 2.x
## What changes were proposed in this pull request? Make some cross-cutting code improvements according to static analysis. These are individually up for discussion since they exist in separate commits that can be reverted. The changes are broadly: - Inner class should be static - Mismatched hashCode/equals - Overflow in compareTo - Unchecked warnings - Misuse of assert, vs junit.assert - get(a) + getOrElse(b) -> getOrElse(a,b) - Array/String .size -> .length (occasionally, -> .isEmpty / .nonEmpty) to avoid implicit conversions - Dead code - tailrec - exists(_ == ) -> contains find + nonEmpty -> exists filter + size -> count - reduce(_+_) -> sum map + flatten -> map The most controversial may be .size -> .length simply because of its size. It is intended to avoid implicits that might be expensive in some places. ## How was the this patch tested? Existing Jenkins unit tests. Author: Sean Owen <[email protected]> Closes #11292 from srowen/SPARK-13423.
1 parent 02b7677 commit e97fc7f

File tree

147 files changed

+345
-293
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

147 files changed

+345
-293
lines changed

common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -132,7 +132,7 @@ public static void tearDown() {
132132
testFile.delete();
133133
}
134134

135-
class FetchResult {
135+
static class FetchResult {
136136
public Set<Integer> successChunks;
137137
public Set<Integer> failedChunks;
138138
public List<ManagedBuffer> buffers;

common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -124,8 +124,8 @@ public StreamManager getStreamManager() {
124124
synchronized (callback1) {
125125
client.sendRpc(ByteBuffer.allocate(0), callback1);
126126
callback1.wait(4 * 1000);
127-
assert (callback1.failure != null);
128-
assert (callback1.failure instanceof IOException);
127+
assertNotNull(callback1.failure);
128+
assertTrue(callback1.failure instanceof IOException);
129129
}
130130
semaphore.release();
131131
}
@@ -167,8 +167,8 @@ public StreamManager getStreamManager() {
167167
synchronized (callback0) {
168168
client0.sendRpc(ByteBuffer.allocate(0), callback0);
169169
callback0.wait(FOREVER);
170-
assert (callback0.failure instanceof IOException);
171-
assert (!client0.isActive());
170+
assertTrue(callback0.failure instanceof IOException);
171+
assertFalse(client0.isActive());
172172
}
173173

174174
// Increment the semaphore and the second request should succeed quickly.
@@ -236,15 +236,15 @@ public StreamManager getStreamManager() {
236236

237237
synchronized (callback1) {
238238
// failed at same time as previous
239-
assert (callback0.failure instanceof IOException);
239+
assertTrue(callback0.failure instanceof IOException);
240240
}
241241
}
242242

243243
/**
244244
* Callback which sets 'success' or 'failure' on completion.
245245
* Additionally notifies all waiters on this callback when invoked.
246246
*/
247-
class TestCallback implements RpcResponseCallback, ChunkReceivedCallback {
247+
static class TestCallback implements RpcResponseCallback, ChunkReceivedCallback {
248248

249249
int successLength = -1;
250250
Throwable failure;

common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -91,7 +91,7 @@ public static void tearDown() {
9191
clientFactory.close();
9292
}
9393

94-
class RpcResult {
94+
static class RpcResult {
9595
public Set<String> successMessages;
9696
public Set<String> errorMessages;
9797
}

common/network-common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727

2828
import com.google.common.collect.Maps;
2929
import org.junit.After;
30+
import org.junit.Assert;
3031
import org.junit.Before;
3132
import org.junit.Test;
3233

@@ -95,7 +96,7 @@ public void run() {
9596
try {
9697
TransportClient client =
9798
factory.createClient(TestUtils.getLocalHost(), server1.getPort());
98-
assert (client.isActive());
99+
assertTrue(client.isActive());
99100
clients.add(client);
100101
} catch (IOException e) {
101102
failed.incrementAndGet();
@@ -115,8 +116,8 @@ public void run() {
115116
attempts[i].join();
116117
}
117118

118-
assert(failed.get() == 0);
119-
assert(clients.size() == maxConnections);
119+
Assert.assertEquals(0, failed.get());
120+
Assert.assertEquals(clients.size(), maxConnections);
120121

121122
for (TransportClient client : clients) {
122123
client.close();

common/network-common/src/test/java/org/apache/spark/network/protocol/MessageWithHeaderSuite.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ public void testByteBufBody() throws Exception {
6565
assertEquals(42, result.readLong());
6666
assertEquals(84, result.readLong());
6767

68-
assert(msg.release());
68+
assertTrue(msg.release());
6969
assertEquals(0, bodyPassedToNettyManagedBuffer.refCnt());
7070
assertEquals(0, header.refCnt());
7171
}
@@ -77,7 +77,7 @@ public void testDeallocateReleasesManagedBuffer() throws Exception {
7777
ByteBuf body = (ByteBuf) managedBuf.convertToNetty();
7878
assertEquals(2, body.refCnt());
7979
MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes());
80-
assert(msg.release());
80+
assertTrue(msg.release());
8181
Mockito.verify(managedBuf, Mockito.times(1)).release();
8282
assertEquals(0, body.refCnt());
8383
}
@@ -94,7 +94,7 @@ private void testFileRegionBody(int totalWrites, int writesPerCall) throws Excep
9494
for (long i = 0; i < 8; i++) {
9595
assertEquals(i, result.readLong());
9696
}
97-
assert(msg.release());
97+
assertTrue(msg.release());
9898
}
9999

100100
private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception {

common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,14 @@ public int hashCode() {
5656
return Objects.hashCode(appId);
5757
}
5858

59+
@Override
60+
public boolean equals(Object o) {
61+
if (!(o instanceof RegisterDriver)) {
62+
return false;
63+
}
64+
return Objects.equal(appId, ((RegisterDriver) o).appId);
65+
}
66+
5967
public static RegisterDriver decode(ByteBuf buf) {
6068
String appId = Encoders.Strings.decode(buf);
6169
return new RegisterDriver(appId);

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -109,7 +109,7 @@ public void afterEach() {
109109
handler.applicationRemoved(APP_ID, false /* cleanupLocalDirs */);
110110
}
111111

112-
class FetchResult {
112+
static class FetchResult {
113113
public Set<String> successBlocks;
114114
public Set<String> failedBlocks;
115115
public List<ManagedBuffer> buffers;

common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -305,7 +305,7 @@ public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
305305
}
306306
}
307307

308-
assert stub != null;
308+
assertNotNull(stub);
309309
stub.when(fetchStarter).createAndStart((String[]) any(), (BlockFetchingListener) anyObject());
310310
String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]);
311311
new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start();

common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -193,7 +193,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty
193193

194194
test("concat") {
195195
def concat(orgin: Seq[String]): String =
196-
if (orgin.exists(_ == null)) null else orgin.mkString
196+
if (orgin.contains(null)) null else orgin.mkString
197197

198198
forAll { (inputs: Seq[String]) =>
199199
assert(UTF8String.concat(inputs.map(toUTF8): _*) === toUTF8(inputs.mkString))

core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -30,7 +30,9 @@ final class ShuffleInMemorySorter {
3030
private static final class SortComparator implements Comparator<PackedRecordPointer> {
3131
@Override
3232
public int compare(PackedRecordPointer left, PackedRecordPointer right) {
33-
return left.getPartitionId() - right.getPartitionId();
33+
int leftId = left.getPartitionId();
34+
int rightId = right.getPartitionId();
35+
return leftId < rightId ? -1 : (leftId > rightId ? 1 : 0);
3436
}
3537
}
3638
private static final SortComparator SORT_COMPARATOR = new SortComparator();

0 commit comments

Comments
 (0)