Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Commit b6ef1f5

Browse files
original-brownbearsrowen
authored andcommitted
[SPARK-21970][CORE] Fix Redundant Throws Declarations in Java Codebase
## What changes were proposed in this pull request? 1. Removing all redundant throws declarations from Java codebase. 2. Removing dead code made visible by this from `ShuffleExternalSorter#closeAndGetSpills` ## How was this patch tested? Build still passes. Author: Armin <[email protected]> Closes apache#19182 from original-brownbear/SPARK-21970.
1 parent 0fa5b7c commit b6ef1f5

File tree

15 files changed

+36
-47
lines changed

15 files changed

+36
-47
lines changed

common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -249,7 +249,7 @@ Index parent() {
249249
* calculated only once, avoiding redundant work when multiple child indices of the
250250
* same parent index exist.
251251
*/
252-
byte[] childPrefix(Object value) throws Exception {
252+
byte[] childPrefix(Object value) {
253253
Preconditions.checkState(parent == null, "Not a parent index.");
254254
return buildKey(name, toParentKey(value));
255255
}
@@ -295,7 +295,7 @@ byte[] end(byte[] prefix) {
295295
}
296296

297297
/** The key for the end marker for entries with the given value. */
298-
byte[] end(byte[] prefix, Object value) throws Exception {
298+
byte[] end(byte[] prefix, Object value) {
299299
checkParent(prefix);
300300
return (parent != null) ? buildKey(false, prefix, name, toKey(value), END_MARKER)
301301
: buildKey(name, toKey(value), END_MARKER);
@@ -313,7 +313,7 @@ byte[] entityKey(byte[] prefix, Object entity) throws Exception {
313313
return entityKey;
314314
}
315315

316-
private void updateCount(WriteBatch batch, byte[] key, long delta) throws Exception {
316+
private void updateCount(WriteBatch batch, byte[] key, long delta) {
317317
long updated = getCount(key) + delta;
318318
if (updated > 0) {
319319
batch.put(key, db.serializer.serialize(updated));
@@ -431,7 +431,7 @@ void remove(
431431
addOrRemove(batch, entity, null, null, naturalKey, prefix);
432432
}
433433

434-
long getCount(byte[] key) throws Exception {
434+
long getCount(byte[] key) {
435435
byte[] data = db.db().get(key);
436436
return data != null ? db.serializer.deserializeLong(data) : 0;
437437
}

common/network-common/src/main/java/org/apache/spark/network/crypto/AuthEngine.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -81,7 +81,7 @@ class AuthEngine implements Closeable {
8181
*
8282
* @return A challenge to be sent the remote side.
8383
*/
84-
ClientChallenge challenge() throws GeneralSecurityException, IOException {
84+
ClientChallenge challenge() throws GeneralSecurityException {
8585
this.authNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE);
8686
SecretKeySpec authKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(),
8787
authNonce, conf.encryptionKeyLength());
@@ -105,7 +105,7 @@ ClientChallenge challenge() throws GeneralSecurityException, IOException {
105105
* @return A response to be sent to the client.
106106
*/
107107
ServerResponse respond(ClientChallenge clientChallenge)
108-
throws GeneralSecurityException, IOException {
108+
throws GeneralSecurityException {
109109

110110
SecretKeySpec authKey = generateKey(clientChallenge.kdf, clientChallenge.iterations,
111111
clientChallenge.nonce, clientChallenge.keyLength);

common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717

1818
package org.apache.spark.network.sasl;
1919

20-
import java.io.IOException;
2120
import java.util.Map;
2221
import javax.security.auth.callback.Callback;
2322
import javax.security.auth.callback.CallbackHandler;
@@ -125,7 +124,7 @@ public synchronized void dispose() {
125124
*/
126125
private class ClientCallbackHandler implements CallbackHandler {
127126
@Override
128-
public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
127+
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
129128

130129
for (Callback callback : callbacks) {
131130
if (callback instanceof NameCallback) {

common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,6 @@
2727
import javax.security.sasl.Sasl;
2828
import javax.security.sasl.SaslException;
2929
import javax.security.sasl.SaslServer;
30-
import java.io.IOException;
3130
import java.nio.charset.StandardCharsets;
3231
import java.util.Map;
3332

@@ -155,7 +154,7 @@ public byte[] unwrap(byte[] data, int offset, int len) throws SaslException {
155154
*/
156155
private class DigestCallbackHandler implements CallbackHandler {
157156
@Override
158-
public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
157+
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
159158
for (Callback callback : callbacks) {
160159
if (callback instanceof NameCallback) {
161160
logger.trace("SASL server callback: setting username");

common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -117,7 +117,7 @@ private void init(String hostToBind, int portToBind) {
117117

118118
bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
119119
@Override
120-
protected void initChannel(SocketChannel ch) throws Exception {
120+
protected void initChannel(SocketChannel ch) {
121121
RpcHandler rpcHandler = appRpcHandler;
122122
for (TransportServerBootstrap bootstrap : bootstraps) {
123123
rpcHandler = bootstrap.doBootstrap(ch, rpcHandler);

common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -121,7 +121,7 @@ private long decodeFrameSize() {
121121
return nextFrameSize;
122122
}
123123

124-
private ByteBuf decodeNext() throws Exception {
124+
private ByteBuf decodeNext() {
125125
long frameSize = decodeFrameSize();
126126
if (frameSize == UNKNOWN_FRAME_SIZE || totalSize < frameSize) {
127127
return null;

common/sketch/src/main/java/org/apache/spark/util/sketch/CountMinSketchImpl.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -365,7 +365,7 @@ private void writeObject(ObjectOutputStream out) throws IOException {
365365
this.writeTo(out);
366366
}
367367

368-
private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
368+
private void readObject(ObjectInputStream in) throws IOException {
369369
this.readFrom0(in);
370370
}
371371
}

core/src/main/java/org/apache/spark/memory/MemoryConsumer.java

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -111,8 +111,6 @@ public void freeArray(LongArray array) {
111111
/**
112112
* Allocate a memory block with at least `required` bytes.
113113
*
114-
* Throws IOException if there is not enough memory.
115-
*
116114
* @throws OutOfMemoryError
117115
*/
118116
protected MemoryBlock allocatePage(long required) {

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

Lines changed: 9 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -140,7 +140,7 @@ final class ShuffleExternalSorter extends MemoryConsumer {
140140
* bytes written should be counted towards shuffle spill metrics rather than
141141
* shuffle write metrics.
142142
*/
143-
private void writeSortedFile(boolean isLastFile) throws IOException {
143+
private void writeSortedFile(boolean isLastFile) {
144144

145145
final ShuffleWriteMetrics writeMetricsToUse;
146146

@@ -325,7 +325,7 @@ public void cleanupResources() {
325325
* array and grows the array if additional space is required. If the required space cannot be
326326
* obtained, then the in-memory data will be spilled to disk.
327327
*/
328-
private void growPointerArrayIfNecessary() throws IOException {
328+
private void growPointerArrayIfNecessary() {
329329
assert(inMemSorter != null);
330330
if (!inMemSorter.hasSpaceForAnotherRecord()) {
331331
long used = inMemSorter.getMemoryUsage();
@@ -406,19 +406,14 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p
406406
* @throws IOException
407407
*/
408408
public SpillInfo[] closeAndGetSpills() throws IOException {
409-
try {
410-
if (inMemSorter != null) {
411-
// Do not count the final file towards the spill count.
412-
writeSortedFile(true);
413-
freeMemory();
414-
inMemSorter.free();
415-
inMemSorter = null;
416-
}
417-
return spills.toArray(new SpillInfo[spills.size()]);
418-
} catch (IOException e) {
419-
cleanupResources();
420-
throw e;
409+
if (inMemSorter != null) {
410+
// Do not count the final file towards the spill count.
411+
writeSortedFile(true);
412+
freeMemory();
413+
inMemSorter.free();
414+
inMemSorter = null;
421415
}
416+
return spills.toArray(new SpillInfo[spills.size()]);
422417
}
423418

424419
}

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -208,7 +208,7 @@ public void write(scala.collection.Iterator<Product2<K, V>> records) throws IOEx
208208
}
209209
}
210210

211-
private void open() throws IOException {
211+
private void open() {
212212
assert (sorter == null);
213213
sorter = new ShuffleExternalSorter(
214214
memoryManager,

0 commit comments

Comments
 (0)