Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -249,7 +249,7 @@ Index parent() {
* calculated only once, avoiding redundant work when multiple child indices of the
* same parent index exist.
*/
byte[] childPrefix(Object value) throws Exception {
byte[] childPrefix(Object value) {
Preconditions.checkState(parent == null, "Not a parent index.");
return buildKey(name, toParentKey(value));
}
Expand Down Expand Up @@ -295,7 +295,7 @@ byte[] end(byte[] prefix) {
}

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

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

long getCount(byte[] key) throws Exception {
long getCount(byte[] key) {
byte[] data = db.db().get(key);
return data != null ? db.serializer.deserializeLong(data) : 0;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ class AuthEngine implements Closeable {
*
* @return A challenge to be sent the remote side.
*/
ClientChallenge challenge() throws GeneralSecurityException, IOException {
ClientChallenge challenge() throws GeneralSecurityException {
this.authNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE);
SecretKeySpec authKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(),
authNonce, conf.encryptionKeyLength());
Expand All @@ -105,7 +105,7 @@ ClientChallenge challenge() throws GeneralSecurityException, IOException {
* @return A response to be sent to the client.
*/
ServerResponse respond(ClientChallenge clientChallenge)
throws GeneralSecurityException, IOException {
throws GeneralSecurityException {

SecretKeySpec authKey = generateKey(clientChallenge.kdf, clientChallenge.iterations,
clientChallenge.nonce, clientChallenge.keyLength);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.network.sasl;

import java.io.IOException;
import java.util.Map;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.CallbackHandler;
Expand Down Expand Up @@ -125,7 +124,7 @@ public synchronized void dispose() {
*/
private class ClientCallbackHandler implements CallbackHandler {
@Override
public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {

for (Callback callback : callbacks) {
if (callback instanceof NameCallback) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import javax.security.sasl.Sasl;
import javax.security.sasl.SaslException;
import javax.security.sasl.SaslServer;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Map;

Expand Down Expand Up @@ -155,7 +154,7 @@ public byte[] unwrap(byte[] data, int offset, int len) throws SaslException {
*/
private class DigestCallbackHandler implements CallbackHandler {
@Override
public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException {
public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
for (Callback callback : callbacks) {
if (callback instanceof NameCallback) {
logger.trace("SASL server callback: setting username");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ private void init(String hostToBind, int portToBind) {

bootstrap.childHandler(new ChannelInitializer<SocketChannel>() {
@Override
protected void initChannel(SocketChannel ch) throws Exception {
protected void initChannel(SocketChannel ch) {
RpcHandler rpcHandler = appRpcHandler;
for (TransportServerBootstrap bootstrap : bootstraps) {
rpcHandler = bootstrap.doBootstrap(ch, rpcHandler);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,7 @@ private long decodeFrameSize() {
return nextFrameSize;
}

private ByteBuf decodeNext() throws Exception {
private ByteBuf decodeNext() {
long frameSize = decodeFrameSize();
if (frameSize == UNKNOWN_FRAME_SIZE || totalSize < frameSize) {
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -365,7 +365,7 @@ private void writeObject(ObjectOutputStream out) throws IOException {
this.writeTo(out);
}

private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
private void readObject(ObjectInputStream in) throws IOException {
this.readFrom0(in);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -111,8 +111,6 @@ public void freeArray(LongArray array) {
/**
* Allocate a memory block with at least `required` bytes.
*
* Throws IOException if there is not enough memory.
*
* @throws OutOfMemoryError
*/
protected MemoryBlock allocatePage(long required) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ final class ShuffleExternalSorter extends MemoryConsumer {
* bytes written should be counted towards shuffle spill metrics rather than
* shuffle write metrics.
*/
private void writeSortedFile(boolean isLastFile) throws IOException {
private void writeSortedFile(boolean isLastFile) {

final ShuffleWriteMetrics writeMetricsToUse;

Expand Down Expand Up @@ -325,7 +325,7 @@ public void cleanupResources() {
* array and grows the array if additional space is required. If the required space cannot be
* obtained, then the in-memory data will be spilled to disk.
*/
private void growPointerArrayIfNecessary() throws IOException {
private void growPointerArrayIfNecessary() {
assert(inMemSorter != null);
if (!inMemSorter.hasSpaceForAnotherRecord()) {
long used = inMemSorter.getMemoryUsage();
Expand Down Expand Up @@ -406,19 +406,14 @@ public void insertRecord(Object recordBase, long recordOffset, int length, int p
* @throws IOException
*/
public SpillInfo[] closeAndGetSpills() throws IOException {
try {
if (inMemSorter != null) {
// Do not count the final file towards the spill count.
writeSortedFile(true);
freeMemory();
inMemSorter.free();
inMemSorter = null;
}
return spills.toArray(new SpillInfo[spills.size()]);
} catch (IOException e) {
cleanupResources();
throw e;
if (inMemSorter != null) {
// Do not count the final file towards the spill count.
writeSortedFile(true);
freeMemory();
inMemSorter.free();
inMemSorter = null;
}
return spills.toArray(new SpillInfo[spills.size()]);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,7 @@ public void write(scala.collection.Iterator<Product2<K, V>> records) throws IOEx
}
}

private void open() throws IOException {
private void open() {
assert (sorter == null);
sorter = new ShuffleExternalSorter(
memoryManager,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -346,7 +346,7 @@ public void cleanupResources() {
* array and grows the array if additional space is required. If the required space cannot be
* obtained, then the in-memory data will be spilled to disk.
*/
private void growPointerArrayIfNecessary() throws IOException {
private void growPointerArrayIfNecessary() {
assert(inMemSorter != null);
if (!inMemSorter.hasSpaceForAnotherRecord()) {
long used = inMemSorter.getMemoryUsage();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ public UnsafeSorterSpillWriter(
}

// Based on DataOutputStream.writeLong.
private void writeLongToBuffer(long v, int offset) throws IOException {
private void writeLongToBuffer(long v, int offset) {
writeBuffer[offset + 0] = (byte)(v >>> 56);
writeBuffer[offset + 1] = (byte)(v >>> 48);
writeBuffer[offset + 2] = (byte)(v >>> 40);
Expand All @@ -91,7 +91,7 @@ private void writeLongToBuffer(long v, int offset) throws IOException {
}

// Based on DataOutputStream.writeInt.
private void writeIntToBuffer(int v, int offset) throws IOException {
private void writeIntToBuffer(int v, int offset) {
writeBuffer[offset + 0] = (byte)(v >>> 24);
writeBuffer[offset + 1] = (byte)(v >>> 16);
writeBuffer[offset + 2] = (byte)(v >>> 8);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ public static void main(String[] args) throws Exception {
FlatMapFunction<LineWithTimestamp, Event> linesToEvents =
new FlatMapFunction<LineWithTimestamp, Event>() {
@Override
public Iterator<Event> call(LineWithTimestamp lineWithTimestamp) throws Exception {
public Iterator<Event> call(LineWithTimestamp lineWithTimestamp) {
ArrayList<Event> eventList = new ArrayList<Event>();
for (String word : lineWithTimestamp.getLine().split(" ")) {
eventList.add(new Event(word, lineWithTimestamp.getTimestamp()));
Expand All @@ -91,8 +91,7 @@ public Iterator<Event> call(LineWithTimestamp lineWithTimestamp) throws Exceptio
MapGroupsWithStateFunction<String, Event, SessionInfo, SessionUpdate> stateUpdateFunc =
new MapGroupsWithStateFunction<String, Event, SessionInfo, SessionUpdate>() {
@Override public SessionUpdate call(
String sessionId, Iterator<Event> events, GroupState<SessionInfo> state)
throws Exception {
String sessionId, Iterator<Event> events, GroupState<SessionInfo> state) {
// If timed out, then remove session and send final update
if (state.hasTimedOut()) {
SessionUpdate finalUpdate = new SessionUpdate(
Expand Down Expand Up @@ -138,7 +137,7 @@ public Iterator<Event> call(LineWithTimestamp lineWithTimestamp) throws Exceptio
Dataset<SessionUpdate> sessionUpdates = events
.groupByKey(
new MapFunction<Event, String>() {
@Override public String call(Event event) throws Exception {
@Override public String call(Event event) {
return event.getSessionId();
}
}, Encoders.STRING())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -605,7 +605,7 @@ private ProcessBuilder createBuilder() throws IOException {
}

// Visible for testing.
String findSparkSubmit() throws IOException {
String findSparkSubmit() {
String script = isWindows() ? "spark-submit.cmd" : "spark-submit";
return join(File.separator, builder.getSparkHome(), "bin", script);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -350,14 +350,13 @@ private void decodeDictionaryIds(
* is guaranteed that num is smaller than the number of values left in the current page.
*/

private void readBooleanBatch(int rowId, int num, WritableColumnVector column)
throws IOException {
private void readBooleanBatch(int rowId, int num, WritableColumnVector column) {
assert(column.dataType() == DataTypes.BooleanType);
defColumn.readBooleans(
num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn);
}

private void readIntBatch(int rowId, int num, WritableColumnVector column) throws IOException {
private void readIntBatch(int rowId, int num, WritableColumnVector column) {
// This is where we implement support for the valid type conversions.
// TODO: implement remaining type conversions
if (column.dataType() == DataTypes.IntegerType || column.dataType() == DataTypes.DateType ||
Expand All @@ -375,7 +374,7 @@ private void readIntBatch(int rowId, int num, WritableColumnVector column) throw
}
}

private void readLongBatch(int rowId, int num, WritableColumnVector column) throws IOException {
private void readLongBatch(int rowId, int num, WritableColumnVector column) {
// This is where we implement support for the valid type conversions.
if (column.dataType() == DataTypes.LongType ||
DecimalType.is64BitDecimalType(column.dataType())) {
Expand All @@ -394,7 +393,7 @@ private void readLongBatch(int rowId, int num, WritableColumnVector column) thro
}
}

private void readFloatBatch(int rowId, int num, WritableColumnVector column) throws IOException {
private void readFloatBatch(int rowId, int num, WritableColumnVector column) {
// This is where we implement support for the valid type conversions.
// TODO: support implicit cast to double?
if (column.dataType() == DataTypes.FloatType) {
Expand All @@ -405,7 +404,7 @@ private void readFloatBatch(int rowId, int num, WritableColumnVector column) thr
}
}

private void readDoubleBatch(int rowId, int num, WritableColumnVector column) throws IOException {
private void readDoubleBatch(int rowId, int num, WritableColumnVector column) {
// This is where we implement support for the valid type conversions.
// TODO: implement remaining type conversions
if (column.dataType() == DataTypes.DoubleType) {
Expand All @@ -416,7 +415,7 @@ private void readDoubleBatch(int rowId, int num, WritableColumnVector column) th
}
}

private void readBinaryBatch(int rowId, int num, WritableColumnVector column) throws IOException {
private void readBinaryBatch(int rowId, int num, WritableColumnVector column) {
// This is where we implement support for the valid type conversions.
// TODO: implement remaining type conversions
VectorizedValuesReader data = (VectorizedValuesReader) dataColumn;
Expand All @@ -441,7 +440,7 @@ private void readFixedLenByteArrayBatch(
int rowId,
int num,
WritableColumnVector column,
int arrayLen) throws IOException {
int arrayLen) {
VectorizedValuesReader data = (VectorizedValuesReader) dataColumn;
// This is where we implement support for the valid type conversions.
// TODO: implement remaining type conversions
Expand Down Expand Up @@ -476,7 +475,7 @@ private void readFixedLenByteArrayBatch(
}
}

private void readPage() throws IOException {
private void readPage() {
DataPage page = pageReader.readPage();
// TODO: Why is this a visitor?
page.accept(new DataPage.Visitor<Void>() {
Expand Down