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 @@ -159,15 +159,21 @@ public void close() throws IOException {
// accurately report the errors when they happen.
RuntimeException error = null;
byte[] dummy = new byte[8];
try {
doCipherOp(encryptor, dummy, true);
} catch (Exception e) {
error = new RuntimeException(e);
if (encryptor != null) {
try {
doCipherOp(Cipher.ENCRYPT_MODE, dummy, true);
} catch (Exception e) {
error = new RuntimeException(e);
}
encryptor = null;
}
try {
doCipherOp(decryptor, dummy, true);
} catch (Exception e) {
error = new RuntimeException(e);
if (decryptor != null) {
try {
doCipherOp(Cipher.DECRYPT_MODE, dummy, true);
} catch (Exception e) {
error = new RuntimeException(e);
}
decryptor = null;
}
random.close();

Expand All @@ -189,11 +195,11 @@ byte[] rawResponse(byte[] challenge) {
}

private byte[] decrypt(byte[] in) throws GeneralSecurityException {
return doCipherOp(decryptor, in, false);
return doCipherOp(Cipher.DECRYPT_MODE, in, false);
}

private byte[] encrypt(byte[] in) throws GeneralSecurityException {
return doCipherOp(encryptor, in, false);
return doCipherOp(Cipher.ENCRYPT_MODE, in, false);
}

private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key)
Expand All @@ -205,11 +211,13 @@ private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key)
byte[] iv = new byte[conf.ivLength()];
System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length));

encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf);
encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv));
CryptoCipher _encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf);
_encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv));
this.encryptor = _encryptor;

decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf);
decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv));
CryptoCipher _decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf);
_decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv));
this.decryptor = _decryptor;
}

/**
Expand Down Expand Up @@ -241,29 +249,52 @@ private SecretKeySpec generateKey(String kdf, int iterations, byte[] salt, int k
return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm());
}

private byte[] doCipherOp(CryptoCipher cipher, byte[] in, boolean isFinal)
private byte[] doCipherOp(int mode, byte[] in, boolean isFinal)
throws GeneralSecurityException {

Preconditions.checkState(cipher != null);
CryptoCipher cipher;
switch (mode) {
case Cipher.ENCRYPT_MODE:
cipher = encryptor;
break;
case Cipher.DECRYPT_MODE:
cipher = decryptor;
break;
default:
throw new IllegalArgumentException(String.valueOf(mode));
}

int scale = 1;
while (true) {
int size = in.length * scale;
byte[] buffer = new byte[size];
try {
int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0)
: cipher.update(in, 0, in.length, buffer, 0);
if (outSize != buffer.length) {
byte[] output = new byte[outSize];
System.arraycopy(buffer, 0, output, 0, output.length);
return output;
} else {
return buffer;
Preconditions.checkState(cipher != null, "Cipher is invalid because of previous error.");

try {
int scale = 1;
while (true) {
int size = in.length * scale;
byte[] buffer = new byte[size];
try {
int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0)
: cipher.update(in, 0, in.length, buffer, 0);
if (outSize != buffer.length) {
byte[] output = new byte[outSize];
System.arraycopy(buffer, 0, output, 0, output.length);
return output;
} else {
return buffer;
}
} catch (ShortBufferException e) {
// Try again with a bigger buffer.
scale *= 2;
}
} catch (ShortBufferException e) {
// Try again with a bigger buffer.
scale *= 2;
}
} catch (InternalError ie) {
// SPARK-25535. The commons-cryto library will throw InternalError if something goes wrong,
// and leave bad state behind in the Java wrappers, so it's not safe to use them afterwards.
if (mode == Cipher.ENCRYPT_MODE) {
this.encryptor = null;
} else {
this.decryptor = null;
}
throw ie;
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,10 +110,12 @@ public void addToChannel(Channel ch) throws IOException {
static class EncryptionHandler extends ChannelOutboundHandlerAdapter {
private final ByteArrayWritableChannel byteChannel;
private final CryptoOutputStream cos;
private boolean isCipherValid;

EncryptionHandler(TransportCipher cipher) throws IOException {
byteChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE);
cos = cipher.createOutputStream(byteChannel);
isCipherValid = true;
}

@Override
Expand All @@ -124,36 +126,61 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)

@VisibleForTesting
EncryptedMessage createEncryptedMessage(Object msg) {
return new EncryptedMessage(cos, msg, byteChannel);
return new EncryptedMessage(this, cos, msg, byteChannel);
}

@Override
public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception {
try {
cos.close();
if (isCipherValid) {
cos.close();
}
} finally {
super.close(ctx, promise);
}
}

/**
* SPARK-25535. Workaround for CRYPTO-141. Avoid further interaction with the underlying cipher
* after an error occurs.
*/
void reportError() {
this.isCipherValid = false;
}

boolean isCipherValid() {
return isCipherValid;
}
}

private static class DecryptionHandler extends ChannelInboundHandlerAdapter {
private final CryptoInputStream cis;
private final ByteArrayReadableChannel byteChannel;
private boolean isCipherValid;

DecryptionHandler(TransportCipher cipher) throws IOException {
byteChannel = new ByteArrayReadableChannel();
cis = cipher.createInputStream(byteChannel);
isCipherValid = true;
}

@Override
public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception {
if (!isCipherValid) {
throw new IOException("Cipher is in invalid state.");
}
byteChannel.feedData((ByteBuf) data);

byte[] decryptedData = new byte[byteChannel.readableBytes()];
int offset = 0;
while (offset < decryptedData.length) {
offset += cis.read(decryptedData, offset, decryptedData.length - offset);
// SPARK-25535: workaround for CRYPTO-141.
try {
offset += cis.read(decryptedData, offset, decryptedData.length - offset);
} catch (InternalError ie) {
isCipherValid = false;
throw ie;
}
}

ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length));
Expand All @@ -162,7 +189,9 @@ public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception
@Override
public void channelInactive(ChannelHandlerContext ctx) throws Exception {
try {
cis.close();
if (isCipherValid) {
cis.close();
}
} finally {
super.channelInactive(ctx);
}
Expand All @@ -175,8 +204,9 @@ static class EncryptedMessage extends AbstractFileRegion {
private final ByteBuf buf;
private final FileRegion region;
private final long count;
private final CryptoOutputStream cos;
private final EncryptionHandler handler;
private long transferred;
private CryptoOutputStream cos;

// Due to streaming issue CRYPTO-125: https://issues.apache.org/jira/browse/CRYPTO-125, it has
// to utilize two helper ByteArrayWritableChannel for streaming. One is used to receive raw data
Expand All @@ -186,9 +216,14 @@ static class EncryptedMessage extends AbstractFileRegion {

private ByteBuffer currentEncrypted;

EncryptedMessage(CryptoOutputStream cos, Object msg, ByteArrayWritableChannel ch) {
EncryptedMessage(
EncryptionHandler handler,
CryptoOutputStream cos,
Object msg,
ByteArrayWritableChannel ch) {
Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion,
"Unrecognized message type: %s", msg.getClass().getName());
this.handler = handler;
this.isByteBuf = msg instanceof ByteBuf;
this.buf = isByteBuf ? (ByteBuf) msg : null;
this.region = isByteBuf ? null : (FileRegion) msg;
Expand Down Expand Up @@ -288,6 +323,9 @@ public long transferTo(WritableByteChannel target, long position) throws IOExcep
}

private void encryptMore() throws IOException {
if (!handler.isCipherValid()) {
throw new IOException("Cipher is in invalid state.");
}
byteRawChannel.reset();

if (isByteBuf) {
Expand All @@ -296,8 +334,14 @@ private void encryptMore() throws IOException {
} else {
region.transferTo(byteRawChannel, region.transferred());
}
cos.write(byteRawChannel.getData(), 0, byteRawChannel.length());
cos.flush();

try {
cos.write(byteRawChannel.getData(), 0, byteRawChannel.length());
cos.flush();
} catch (InternalError ie) {
handler.reportError();
throw ie;
}

currentEncrypted = ByteBuffer.wrap(byteEncChannel.getData(),
0, byteEncChannel.length());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,13 @@
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
import java.util.Arrays;
import java.util.Map;
import java.security.InvalidKeyException;
import java.util.Random;

import static java.nio.charset.StandardCharsets.UTF_8;

import com.google.common.collect.ImmutableMap;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.channel.FileRegion;
Expand Down Expand Up @@ -189,4 +192,18 @@ public Long answer(InvocationOnMock invocationOnMock) throws Throwable {
server.close();
}
}

@Test(expected = InvalidKeyException.class)
public void testBadKeySize() throws Exception {
Map<String, String> mconf = ImmutableMap.of("spark.network.crypto.keyLength", "42");
TransportConf conf = new TransportConf("rpc", new MapConfigProvider(mconf));

try (AuthEngine engine = new AuthEngine("appId", "secret", conf)) {
engine.challenge();
fail("Should have failed to create challenge message.");

// Call close explicitly to make sure it's idempotent.
engine.close();
}
}
}
Loading