HADOOP-10628. Javadoc and few code style improvement for Crypto input and output streams. (yliu via clamb)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1598429 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b20180ffa6
commit
2e5ae1aad7
|
@ -11,6 +11,9 @@ fs-encryption (Unreleased)
|
|||
HADOOP-10603. Crypto input and output streams implementing Hadoop stream
|
||||
interfaces. (Yi Liu and Charles Lamb)
|
||||
|
||||
HADOOP-10628. Javadoc and few code style improvement for Crypto
|
||||
input and output streams. (yliu via clamb)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
|
|
@ -40,15 +40,15 @@ public abstract class AESCTRCryptoCodec extends CryptoCodec {
|
|||
}
|
||||
|
||||
/**
|
||||
* IV is produced by combining initial IV and the counter using addition.
|
||||
* IV length should be the same as {@link #AES_BLOCK_SIZE}
|
||||
* The IV is produced by adding the initial IV to the counter. IV length
|
||||
* should be the same as {@link #AES_BLOCK_SIZE}
|
||||
*/
|
||||
@Override
|
||||
public void calculateIV(byte[] initIV, long counter, byte[] IV) {
|
||||
Preconditions.checkArgument(initIV.length == AES_BLOCK_SIZE);
|
||||
Preconditions.checkArgument(IV.length == AES_BLOCK_SIZE);
|
||||
|
||||
ByteBuffer buf = ByteBuffer.wrap(IV);
|
||||
final ByteBuffer buf = ByteBuffer.wrap(IV);
|
||||
buf.put(initIV);
|
||||
buf.order(ByteOrder.BIG_ENDIAN);
|
||||
counter += buf.getLong(AES_BLOCK_SIZE - 8);
|
||||
|
|
|
@ -41,37 +41,37 @@ public abstract class CryptoCodec implements Configurable {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get block size of a block cipher.
|
||||
* Get the block size of a block cipher.
|
||||
* For different algorithms, the block size may be different.
|
||||
* @return int block size
|
||||
* @return int the block size
|
||||
*/
|
||||
public abstract int getAlgorithmBlockSize();
|
||||
|
||||
/**
|
||||
* Get a {@link #org.apache.hadoop.crypto.Encryptor}.
|
||||
* @return Encryptor
|
||||
* Get an {@link #org.apache.hadoop.crypto.Encryptor}.
|
||||
* @return Encryptor the encryptor
|
||||
*/
|
||||
public abstract Encryptor getEncryptor() throws GeneralSecurityException;
|
||||
|
||||
/**
|
||||
* Get a {@link #org.apache.hadoop.crypto.Decryptor}.
|
||||
* @return Decryptor
|
||||
* @return Decryptor the decryptor
|
||||
*/
|
||||
public abstract Decryptor getDecryptor() throws GeneralSecurityException;
|
||||
|
||||
/**
|
||||
* This interface is only for Counter (CTR) mode. Typically calculating
|
||||
* IV(Initialization Vector) is up to Encryptor or Decryptor, for
|
||||
* example {@link #javax.crypto.Cipher} will maintain encryption context
|
||||
* internally when do encryption/decryption continuously using its
|
||||
* This interface is only for Counter (CTR) mode. Generally the Encryptor
|
||||
* or Decryptor calculates the IV and maintain encryption context internally.
|
||||
* For example a {@link #javax.crypto.Cipher} will maintain its encryption
|
||||
* context internally when we do encryption/decryption using the
|
||||
* Cipher#update interface.
|
||||
* <p/>
|
||||
* In Hadoop, multiple nodes may read splits of a file, so decrypting of
|
||||
* file is not continuous, even for encrypting may be not continuous. For
|
||||
* each part, we need to calculate the counter through file position.
|
||||
* Encryption/Decryption is not always on the entire file. For example,
|
||||
* in Hadoop, a node may only decrypt a portion of a file (i.e. a split).
|
||||
* In these situations, the counter is derived from the file position.
|
||||
* <p/>
|
||||
* Typically IV for a file position is produced by combining initial IV and
|
||||
* the counter using any lossless operation (concatenation, addition, or XOR).
|
||||
* The IV can be calculated by combining the initial IV and the counter with
|
||||
* a lossless operation (concatenation, addition, or XOR).
|
||||
* @see http://en.wikipedia.org/wiki/Block_cipher_mode_of_operation#Counter_.28CTR.29
|
||||
*
|
||||
* @param initIV initial IV
|
||||
|
|
|
@ -63,26 +63,30 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
private static final byte[] oneByteBuf = new byte[1];
|
||||
private final CryptoCodec codec;
|
||||
private final Decryptor decryptor;
|
||||
|
||||
/**
|
||||
* Input data buffer. The data starts at inBuffer.position() and ends at
|
||||
* to inBuffer.limit().
|
||||
*/
|
||||
private ByteBuffer inBuffer;
|
||||
|
||||
/**
|
||||
* The decrypted data buffer. The data starts at outBuffer.position() and
|
||||
* ends at outBuffer.limit();
|
||||
*/
|
||||
private ByteBuffer outBuffer;
|
||||
private long streamOffset = 0; // Underlying stream offset.
|
||||
|
||||
/**
|
||||
* Whether underlying stream supports
|
||||
* Whether the underlying stream supports
|
||||
* {@link #org.apache.hadoop.fs.ByteBufferReadable}
|
||||
*/
|
||||
private Boolean usingByteBufferRead = null;
|
||||
|
||||
/**
|
||||
* Padding = pos%(algorithm blocksize); Padding is put into {@link #inBuffer}
|
||||
* before any other data goes in. The purpose of padding is to put input data
|
||||
* at proper position.
|
||||
* before any other data goes in. The purpose of padding is to put the input
|
||||
* data at proper position.
|
||||
*/
|
||||
private byte padding;
|
||||
private boolean closed;
|
||||
|
@ -144,14 +148,15 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
return 0;
|
||||
}
|
||||
|
||||
int remaining = outBuffer.remaining();
|
||||
final int remaining = outBuffer.remaining();
|
||||
if (remaining > 0) {
|
||||
int n = Math.min(len, remaining);
|
||||
outBuffer.get(b, off, n);
|
||||
return n;
|
||||
} else {
|
||||
int n = 0;
|
||||
/**
|
||||
|
||||
/*
|
||||
* Check whether the underlying stream is {@link ByteBufferReadable},
|
||||
* it can avoid bytes copy.
|
||||
*/
|
||||
|
@ -186,11 +191,11 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
}
|
||||
}
|
||||
|
||||
// Read data from underlying stream.
|
||||
/** Read data from underlying stream. */
|
||||
private int readFromUnderlyingStream() throws IOException {
|
||||
int toRead = inBuffer.remaining();
|
||||
byte[] tmp = getTmpBuf();
|
||||
int n = in.read(tmp, 0, toRead);
|
||||
final int toRead = inBuffer.remaining();
|
||||
final byte[] tmp = getTmpBuf();
|
||||
final int n = in.read(tmp, 0, toRead);
|
||||
if (n > 0) {
|
||||
inBuffer.put(tmp, 0, n);
|
||||
}
|
||||
|
@ -221,19 +226,19 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
inBuffer.clear();
|
||||
outBuffer.flip();
|
||||
if (padding > 0) {
|
||||
/**
|
||||
* The plain text and cipher text have 1:1 mapping, they start at same
|
||||
* position.
|
||||
/*
|
||||
* The plain text and cipher text have a 1:1 mapping, they start at the
|
||||
* same position.
|
||||
*/
|
||||
outBuffer.position(padding);
|
||||
padding = 0;
|
||||
}
|
||||
if (decryptor.isContextReset()) {
|
||||
/**
|
||||
* Typically we will not get here. To improve performance in CTR mode,
|
||||
* we rely on the decryptor maintaining context, for example calculating
|
||||
* the counter. Unfortunately, some bad implementations can't maintain
|
||||
* context so we need to re-init after doing decryption.
|
||||
/*
|
||||
* This code is generally not executed since the decryptor usually
|
||||
* maintains decryption context (e.g. the counter) internally. However,
|
||||
* some implementations can't maintain context so a re-init is necessary
|
||||
* after each decryption call.
|
||||
*/
|
||||
updateDecryptor();
|
||||
}
|
||||
|
@ -243,7 +248,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
* Update the {@link #decryptor}. Calculate the counter and {@link #padding}.
|
||||
*/
|
||||
private void updateDecryptor() throws IOException {
|
||||
long counter = streamOffset / codec.getAlgorithmBlockSize();
|
||||
final long counter = streamOffset / codec.getAlgorithmBlockSize();
|
||||
padding = (byte)(streamOffset % codec.getAlgorithmBlockSize());
|
||||
inBuffer.position(padding); // Set proper position for input data.
|
||||
codec.calculateIV(initIV, counter, iv);
|
||||
|
@ -251,8 +256,8 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
}
|
||||
|
||||
/**
|
||||
* Reset the underlying stream offset; and clear {@link #inBuffer} and
|
||||
* {@link #outBuffer}. Typically this happens when doing {@link #seek(long)}
|
||||
* Reset the underlying stream offset, and clear {@link #inBuffer} and
|
||||
* {@link #outBuffer}. This Typically happens during {@link #seek(long)}
|
||||
* or {@link #skip(long)}.
|
||||
*/
|
||||
private void resetStreamOffset(long offset) throws IOException {
|
||||
|
@ -274,30 +279,29 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
closed = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Free the direct buffer manually.
|
||||
*/
|
||||
/** Forcibly free the direct buffer. */
|
||||
private void freeBuffers() {
|
||||
sun.misc.Cleaner inBufferCleaner =
|
||||
final sun.misc.Cleaner inBufferCleaner =
|
||||
((sun.nio.ch.DirectBuffer) inBuffer).cleaner();
|
||||
inBufferCleaner.clean();
|
||||
sun.misc.Cleaner outBufferCleaner =
|
||||
final sun.misc.Cleaner outBufferCleaner =
|
||||
((sun.nio.ch.DirectBuffer) outBuffer).cleaner();
|
||||
outBufferCleaner.clean();
|
||||
}
|
||||
|
||||
// Positioned read.
|
||||
/** Positioned read. */
|
||||
@Override
|
||||
public int read(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
checkStream();
|
||||
try {
|
||||
int n = ((PositionedReadable) in).read(position, buffer, offset, length);
|
||||
final int n = ((PositionedReadable) in).read(position, buffer, offset,
|
||||
length);
|
||||
if (n > 0) {
|
||||
/**
|
||||
/*
|
||||
* Since this operation does not change the current offset of a file,
|
||||
* streamOffset should be not changed and we need to restore the
|
||||
* decryptor and outBuffer after decryption.
|
||||
* streamOffset should not be changed. We need to restore the decryptor
|
||||
* and outBuffer after decryption.
|
||||
*/
|
||||
decrypt(position, buffer, offset, length);
|
||||
}
|
||||
|
@ -310,24 +314,23 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
}
|
||||
|
||||
/**
|
||||
* Decrypt given length of data in buffer: start from offset.
|
||||
* Output is also buffer and start from same offset. Restore the
|
||||
* {@link #decryptor} and {@link #outBuffer} after decryption.
|
||||
* Decrypt length bytes in buffer starting at offset. Output is also put
|
||||
* into buffer starting at offset. Restore the {@link #decryptor} and
|
||||
* {@link #outBuffer} after the decryption.
|
||||
*/
|
||||
private void decrypt(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
|
||||
byte[] tmp = getTmpBuf();
|
||||
final byte[] tmp = getTmpBuf();
|
||||
int unread = outBuffer.remaining();
|
||||
if (unread > 0) { // Cache outBuffer
|
||||
outBuffer.get(tmp, 0, unread);
|
||||
}
|
||||
long curOffset = streamOffset;
|
||||
final long curOffset = streamOffset;
|
||||
resetStreamOffset(position);
|
||||
|
||||
int n = 0;
|
||||
while (n < length) {
|
||||
int toDecrypt = Math.min(length - n, inBuffer.remaining());
|
||||
final int toDecrypt = Math.min(length - n, inBuffer.remaining());
|
||||
inBuffer.put(buffer, offset + n, toDecrypt);
|
||||
// Do decryption
|
||||
decrypt();
|
||||
|
@ -344,7 +347,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
}
|
||||
}
|
||||
|
||||
// Positioned read fully.
|
||||
/** Positioned read fully. */
|
||||
@Override
|
||||
public void readFully(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
|
@ -352,9 +355,9 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
try {
|
||||
((PositionedReadable) in).readFully(position, buffer, offset, length);
|
||||
if (length > 0) {
|
||||
/**
|
||||
* Since this operation does not change the current offset of a file,
|
||||
* streamOffset should be not changed and we need to restore the decryptor
|
||||
/*
|
||||
* Since this operation does not change the current offset of the file,
|
||||
* streamOffset should not be changed. We need to restore the decryptor
|
||||
* and outBuffer after decryption.
|
||||
*/
|
||||
decrypt(position, buffer, offset, length);
|
||||
|
@ -370,13 +373,16 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
readFully(position, buffer, 0, buffer.length);
|
||||
}
|
||||
|
||||
// Seek to a position.
|
||||
/** Seek to a position. */
|
||||
@Override
|
||||
public void seek(long pos) throws IOException {
|
||||
Preconditions.checkArgument(pos >= 0, "Cannot seek to negative offset.");
|
||||
checkStream();
|
||||
try {
|
||||
// If target pos we have already read and decrypt.
|
||||
/*
|
||||
* If data of target pos in the underlying stream has already been read
|
||||
* and decrypted in outBuffer, we just need to re-position outBuffer.
|
||||
*/
|
||||
if (pos <= streamOffset && pos >= (streamOffset - outBuffer.remaining())) {
|
||||
int forward = (int) (pos - (streamOffset - outBuffer.remaining()));
|
||||
if (forward > 0) {
|
||||
|
@ -392,7 +398,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
}
|
||||
}
|
||||
|
||||
// Skip n bytes
|
||||
/** Skip n bytes */
|
||||
@Override
|
||||
public long skip(long n) throws IOException {
|
||||
Preconditions.checkArgument(n >= 0, "Negative skip length.");
|
||||
|
@ -405,11 +411,11 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
outBuffer.position(pos);
|
||||
return n;
|
||||
} else {
|
||||
/**
|
||||
/*
|
||||
* Subtract outBuffer.remaining() to see how many bytes we need to
|
||||
* skip in underlying stream. We get real skipped bytes number of
|
||||
* underlying stream then add outBuffer.remaining() to get skipped
|
||||
* bytes number from user's view.
|
||||
* skip in the underlying stream. Add outBuffer.remaining() to the
|
||||
* actual number of skipped bytes in the underlying stream to get the
|
||||
* number of skipped bytes from the user's point of view.
|
||||
*/
|
||||
n -= outBuffer.remaining();
|
||||
long skipped = in.skip(n);
|
||||
|
@ -423,7 +429,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
}
|
||||
}
|
||||
|
||||
// Get underlying stream position.
|
||||
/** Get underlying stream position. */
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
checkStream();
|
||||
|
@ -431,16 +437,16 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
return streamOffset - outBuffer.remaining();
|
||||
}
|
||||
|
||||
// ByteBuffer read.
|
||||
/** ByteBuffer read. */
|
||||
@Override
|
||||
public int read(ByteBuffer buf) throws IOException {
|
||||
checkStream();
|
||||
if (in instanceof ByteBufferReadable) {
|
||||
int unread = outBuffer.remaining();
|
||||
final int unread = outBuffer.remaining();
|
||||
if (unread > 0) { // Have unread decrypted data in buffer.
|
||||
int toRead = buf.remaining();
|
||||
if (toRead <= unread) {
|
||||
int limit = outBuffer.limit();
|
||||
final int limit = outBuffer.limit();
|
||||
outBuffer.limit(outBuffer.position() + toRead);
|
||||
buf.put(outBuffer);
|
||||
outBuffer.limit(limit);
|
||||
|
@ -450,8 +456,8 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
}
|
||||
}
|
||||
|
||||
int pos = buf.position();
|
||||
int n = ((ByteBufferReadable) in).read(buf);
|
||||
final int pos = buf.position();
|
||||
final int n = ((ByteBufferReadable) in).read(buf);
|
||||
if (n > 0) {
|
||||
streamOffset += n; // Read n bytes
|
||||
decrypt(buf, n, pos);
|
||||
|
@ -470,8 +476,8 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
*/
|
||||
private void decrypt(ByteBuffer buf, int n, int start)
|
||||
throws IOException {
|
||||
int pos = buf.position();
|
||||
int limit = buf.limit();
|
||||
final int pos = buf.position();
|
||||
final int limit = buf.limit();
|
||||
int len = 0;
|
||||
while (len < n) {
|
||||
buf.position(start + len);
|
||||
|
@ -535,13 +541,13 @@ public class CryptoInputStream extends FilterInputStream implements
|
|||
((Seekable) in).seek(getPos());
|
||||
resetStreamOffset(getPos());
|
||||
}
|
||||
ByteBuffer buffer = ((HasEnhancedByteBufferAccess) in).
|
||||
final ByteBuffer buffer = ((HasEnhancedByteBufferAccess) in).
|
||||
read(bufferPool, maxLength, opts);
|
||||
if (buffer != null) {
|
||||
int n = buffer.remaining();
|
||||
final int n = buffer.remaining();
|
||||
if (n > 0) {
|
||||
streamOffset += buffer.remaining(); // Read n bytes
|
||||
int pos = buffer.position();
|
||||
final int pos = buffer.position();
|
||||
decrypt(buffer, n, pos);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,17 +52,20 @@ public class CryptoOutputStream extends FilterOutputStream implements
|
|||
private static final byte[] oneByteBuf = new byte[1];
|
||||
private final CryptoCodec codec;
|
||||
private final Encryptor encryptor;
|
||||
|
||||
/**
|
||||
* Input data buffer. The data starts at inBuffer.position() and ends at
|
||||
* inBuffer.limit().
|
||||
*/
|
||||
private ByteBuffer inBuffer;
|
||||
|
||||
/**
|
||||
* Encrypted data buffer. The data starts at outBuffer.position() and ends at
|
||||
* outBuffer.limit();
|
||||
*/
|
||||
private ByteBuffer outBuffer;
|
||||
private long streamOffset = 0; // Underlying stream offset.
|
||||
|
||||
/**
|
||||
* Padding = pos%(algorithm blocksize); Padding is put into {@link #inBuffer}
|
||||
* before any other data goes in. The purpose of padding is to put input data
|
||||
|
@ -134,7 +137,7 @@ public class CryptoOutputStream extends FilterOutputStream implements
|
|||
throw new IndexOutOfBoundsException();
|
||||
}
|
||||
while (len > 0) {
|
||||
int remaining = inBuffer.remaining();
|
||||
final int remaining = inBuffer.remaining();
|
||||
if (len < remaining) {
|
||||
inBuffer.put(b, off, len);
|
||||
len = 0;
|
||||
|
@ -163,15 +166,16 @@ public class CryptoOutputStream extends FilterOutputStream implements
|
|||
inBuffer.clear();
|
||||
outBuffer.flip();
|
||||
if (padding > 0) {
|
||||
/**
|
||||
* The plain text and cipher text have 1:1 mapping, they start at same
|
||||
* position.
|
||||
/*
|
||||
* The plain text and cipher text have a 1:1 mapping, they start at the
|
||||
* same position.
|
||||
*/
|
||||
outBuffer.position(padding);
|
||||
padding = 0;
|
||||
}
|
||||
int len = outBuffer.remaining();
|
||||
/**
|
||||
final int len = outBuffer.remaining();
|
||||
|
||||
/*
|
||||
* If underlying stream supports {@link ByteBuffer} write in future, needs
|
||||
* refine here.
|
||||
*/
|
||||
|
@ -181,12 +185,11 @@ public class CryptoOutputStream extends FilterOutputStream implements
|
|||
|
||||
streamOffset += len;
|
||||
if (encryptor.isContextReset()) {
|
||||
/**
|
||||
* We will generally not get here. For CTR mode, to improve
|
||||
* performance, we rely on the encryptor maintaining context, for
|
||||
* example to calculate the counter. But some bad implementations
|
||||
* can't maintain context, and need us to re-init after doing
|
||||
* encryption.
|
||||
/*
|
||||
* This code is generally not executed since the encryptor usually
|
||||
* maintains encryption context (e.g. the counter) internally. However,
|
||||
* some implementations can't maintain context so a re-init is necessary
|
||||
* after each encryption call.
|
||||
*/
|
||||
updateEncryptor();
|
||||
}
|
||||
|
@ -196,7 +199,7 @@ public class CryptoOutputStream extends FilterOutputStream implements
|
|||
* Update the {@link #encryptor}: calculate counter and {@link #padding}.
|
||||
*/
|
||||
private void updateEncryptor() throws IOException {
|
||||
long counter = streamOffset / codec.getAlgorithmBlockSize();
|
||||
final long counter = streamOffset / codec.getAlgorithmBlockSize();
|
||||
padding = (byte)(streamOffset % codec.getAlgorithmBlockSize());
|
||||
inBuffer.position(padding); // Set proper position for input data.
|
||||
codec.calculateIV(initIV, counter, iv);
|
||||
|
@ -222,21 +225,19 @@ public class CryptoOutputStream extends FilterOutputStream implements
|
|||
closed = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Free the direct buffer manually.
|
||||
*/
|
||||
/** Forcibly free the direct buffer. */
|
||||
private void freeBuffers() {
|
||||
sun.misc.Cleaner inBufferCleaner =
|
||||
final sun.misc.Cleaner inBufferCleaner =
|
||||
((sun.nio.ch.DirectBuffer) inBuffer).cleaner();
|
||||
inBufferCleaner.clean();
|
||||
sun.misc.Cleaner outBufferCleaner =
|
||||
final sun.misc.Cleaner outBufferCleaner =
|
||||
((sun.nio.ch.DirectBuffer) outBuffer).cleaner();
|
||||
outBufferCleaner.clean();
|
||||
}
|
||||
|
||||
/**
|
||||
* To flush, we need to encrypt the data in buffer and write to underlying
|
||||
* stream, then do the flush.
|
||||
* To flush, we need to encrypt the data in the buffer and write to the
|
||||
* underlying stream, then do the flush.
|
||||
*/
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
public interface Decryptor {
|
||||
|
||||
/**
|
||||
* Initialize the decryptor, the internal decryption context will be
|
||||
* Initialize the decryptor and the internal decryption context.
|
||||
* reset.
|
||||
* @param key decryption key.
|
||||
* @param iv decryption initialization vector
|
||||
|
@ -37,37 +37,34 @@ public interface Decryptor {
|
|||
public void init(byte[] key, byte[] iv) throws IOException;
|
||||
|
||||
/**
|
||||
* Indicate whether decryption context is reset.
|
||||
* Indicate whether the decryption context is reset.
|
||||
* <p/>
|
||||
* It's useful for some mode like CTR which requires different IV for
|
||||
* different parts of data. Usually decryptor can maintain the context
|
||||
* internally such as calculating IV/counter, then continue a multiple-part
|
||||
* decryption operation without reinit the decryptor using key and the new
|
||||
* IV. For mode like CTR, if context is reset after each decryption, the
|
||||
* decryptor should be reinit before each operation, that's not efficient.
|
||||
* Certain modes, like CTR, require a different IV depending on the
|
||||
* position in the stream. Generally, the decryptor maintains any necessary
|
||||
* context for calculating the IV and counter so that no reinit is necessary
|
||||
* during the decryption. Reinit before each operation is inefficient.
|
||||
* @return boolean whether context is reset.
|
||||
*/
|
||||
public boolean isContextReset();
|
||||
|
||||
/**
|
||||
* This exposes a direct interface for record decryption with direct byte
|
||||
* buffers.
|
||||
* This presents a direct interface decrypting with direct ByteBuffers.
|
||||
* <p/>
|
||||
* The decrypt() function need not always consume the buffers provided,
|
||||
* it will need to be called multiple times to decrypt an entire buffer
|
||||
* and the object will hold the decryption context internally.
|
||||
* This function does not always decrypt the entire buffer and may potentially
|
||||
* need to be called multiple times to process an entire buffer. The object
|
||||
* may hold the decryption context internally.
|
||||
* <p/>
|
||||
* Some implementation may need enough space in the destination buffer to
|
||||
* decrypt an entire input.
|
||||
* Some implementations may require sufficient space in the destination
|
||||
* buffer to decrypt the entire input buffer.
|
||||
* <p/>
|
||||
* The end result will move inBuffer.position() by the bytes-read and
|
||||
* outBuffer.position() by the bytes-written. It should not modify the
|
||||
* inBuffer.limit() or outBuffer.limit() to maintain consistency of operation.
|
||||
* Upon return, inBuffer.position() will be advanced by the number of bytes
|
||||
* read and outBuffer.position() by bytes written. Implementations should
|
||||
* not modify inBuffer.limit() and outBuffer.limit().
|
||||
* <p/>
|
||||
* @param inBuffer in direct {@link ByteBuffer} for reading from. Requires
|
||||
* inBuffer != null and inBuffer.remaining() > 0
|
||||
* @param outBuffer out direct {@link ByteBuffer} for storing the results
|
||||
* into. Requires outBuffer != null and outBuffer.remaining() > 0
|
||||
* @param inBuffer a direct {@link ByteBuffer} to read from. inBuffer may
|
||||
* not be null and inBuffer.remaining() must be > 0
|
||||
* @param outBuffer a direct {@link ByteBuffer} to write to. outBuffer may
|
||||
* not be null and outBuffer.remaining() must be > 0
|
||||
* @throws IOException if decryption fails
|
||||
*/
|
||||
public void decrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
|
||||
|
|
|
@ -28,8 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
public interface Encryptor {
|
||||
|
||||
/**
|
||||
* Initialize the encryptor, the internal encryption context will be
|
||||
* reset.
|
||||
* Initialize the encryptor and the internal encryption context.
|
||||
* @param key encryption key.
|
||||
* @param iv encryption initialization vector
|
||||
* @throws IOException if initialization fails
|
||||
|
@ -37,37 +36,34 @@ public interface Encryptor {
|
|||
public void init(byte[] key, byte[] iv) throws IOException;
|
||||
|
||||
/**
|
||||
* Indicate whether encryption context is reset.
|
||||
* Indicate whether the encryption context is reset.
|
||||
* <p/>
|
||||
* It's useful for some mode like CTR which requires different IV for
|
||||
* different parts of data. Usually encryptor can maintain the context
|
||||
* internally such as calculating IV/counter, then continue a multiple-part
|
||||
* encryption operation without reinit the encryptor using key and the new
|
||||
* IV. For mode like CTR, if context is reset after each encryption, the
|
||||
* encryptor should be reinit before each operation, that's not efficient.
|
||||
* Certain modes, like CTR, require a different IV depending on the
|
||||
* position in the stream. Generally, the encryptor maintains any necessary
|
||||
* context for calculating the IV and counter so that no reinit is necessary
|
||||
* during the encryption. Reinit before each operation is inefficient.
|
||||
* @return boolean whether context is reset.
|
||||
*/
|
||||
public boolean isContextReset();
|
||||
|
||||
/**
|
||||
* This exposes a direct interface for record encryption with direct byte
|
||||
* buffers.
|
||||
* This presents a direct interface encrypting with direct ByteBuffers.
|
||||
* <p/>
|
||||
* The encrypt() function need not always consume the buffers provided,
|
||||
* it will need to be called multiple times to encrypt an entire buffer
|
||||
* and the object will hold the encryption context internally.
|
||||
* This function does not always encrypt the entire buffer and may potentially
|
||||
* need to be called multiple times to process an entire buffer. The object
|
||||
* may hold the encryption context internally.
|
||||
* <p/>
|
||||
* Some implementation may need enough space in the destination buffer to
|
||||
* encrypt an entire input.
|
||||
* Some implementations may require sufficient space in the destination
|
||||
* buffer to encrypt the entire input buffer.
|
||||
* <p/>
|
||||
* The end result will move inBuffer.position() by the bytes-read and
|
||||
* outBuffer.position() by the bytes-written. It should not modify the
|
||||
* inBuffer.limit() or outBuffer.limit() to maintain consistency of operation.
|
||||
* Upon return, inBuffer.position() will be advanced by the number of bytes
|
||||
* read and outBuffer.position() by bytes written. Implementations should
|
||||
* not modify inBuffer.limit() and outBuffer.limit().
|
||||
* <p/>
|
||||
* @param inBuffer in direct {@link ByteBuffer} for reading from. Requires
|
||||
* inBuffer != null and inBuffer.remaining() > 0
|
||||
* @param outBuffer out direct {@link ByteBuffer} for storing the results
|
||||
* into. Requires outBuffer != null and outBuffer.remaining() > 0
|
||||
* @param inBuffer a direct {@link ByteBuffer} to read from. inBuffer may
|
||||
* not be null and inBuffer.remaining() must be > 0
|
||||
* @param outBuffer a direct {@link ByteBuffer} to write to. outBuffer may
|
||||
* not be null and outBuffer.remaining() must be > 0
|
||||
* @throws IOException if encryption fails
|
||||
*/
|
||||
public void encrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
|
||||
|
|
|
@ -53,8 +53,8 @@ public class JCEAESCTRDecryptor implements Decryptor {
|
|||
}
|
||||
|
||||
/**
|
||||
* For AES-CTR, will consume all input data and needs enough space in the
|
||||
* destination buffer to decrypt entire input data.
|
||||
* AES-CTR will consume all of the input data. It requires enough space in
|
||||
* the destination buffer to decrypt entire input buffer.
|
||||
*/
|
||||
@Override
|
||||
public void decrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
|
||||
|
|
|
@ -53,8 +53,8 @@ public class JCEAESCTREncryptor implements Encryptor {
|
|||
}
|
||||
|
||||
/**
|
||||
* For AES-CTR, will consume all input data and needs enough space in the
|
||||
* destination buffer to encrypt entire input data.
|
||||
* AES-CTR will consume all of the input data. It requires enough space in
|
||||
* the destination buffer to encrypt entire input buffer.
|
||||
*/
|
||||
@Override
|
||||
public void encrypt(ByteBuffer inBuffer, ByteBuffer outBuffer)
|
||||
|
|
|
@ -1370,8 +1370,7 @@
|
|||
<name>hadoop.security.crypto.buffer.size</name>
|
||||
<value>8192</value>
|
||||
<description>
|
||||
The buffer size used in Crypto InputStream and OutputStream, and default
|
||||
value is 8192.
|
||||
The buffer size used by CryptoInputStream and CryptoOutputStream.
|
||||
</description>
|
||||
</property>
|
||||
</configuration>
|
||||
|
|
|
@ -42,7 +42,7 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
|
||||
public abstract class CryptoStreamsTestBase {
|
||||
protected static final Log LOG= LogFactory.getLog(
|
||||
protected static final Log LOG = LogFactory.getLog(
|
||||
CryptoStreamsTestBase.class);
|
||||
|
||||
protected static CryptoCodec codec;
|
||||
|
@ -60,13 +60,13 @@ public abstract class CryptoStreamsTestBase {
|
|||
@Before
|
||||
public void setUp() throws IOException {
|
||||
// Generate data
|
||||
int seed = new Random().nextInt();
|
||||
DataOutputBuffer dataBuf = new DataOutputBuffer();
|
||||
RandomDatum.Generator generator = new RandomDatum.Generator(seed);
|
||||
for(int i=0; i < count; ++i) {
|
||||
final int seed = new Random().nextInt();
|
||||
final DataOutputBuffer dataBuf = new DataOutputBuffer();
|
||||
final RandomDatum.Generator generator = new RandomDatum.Generator(seed);
|
||||
for(int i = 0; i < count; ++i) {
|
||||
generator.next();
|
||||
RandomDatum key = generator.getKey();
|
||||
RandomDatum value = generator.getValue();
|
||||
final RandomDatum key = generator.getKey();
|
||||
final RandomDatum value = generator.getValue();
|
||||
|
||||
key.write(dataBuf);
|
||||
value.write(dataBuf);
|
||||
|
@ -114,9 +114,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
protected abstract InputStream getInputStream(int bufferSize, byte[] key,
|
||||
byte[] iv) throws IOException;
|
||||
|
||||
/**
|
||||
* Test crypto reading with different buffer size.
|
||||
*/
|
||||
/** Test crypto reading with different buffer size. */
|
||||
@Test(timeout=120000)
|
||||
public void testRead() throws Exception {
|
||||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
|
@ -148,9 +146,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
in.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test crypto with different IV.
|
||||
*/
|
||||
/** Test crypto with different IV. */
|
||||
@Test(timeout=120000)
|
||||
public void testCryptoIV() throws Exception {
|
||||
byte[] iv1 = iv.clone();
|
||||
|
@ -202,7 +198,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
private void syncableCheck() throws IOException {
|
||||
OutputStream out = getOutputStream(smallBufferSize);
|
||||
try {
|
||||
int bytesWritten = dataLen/3;
|
||||
int bytesWritten = dataLen / 3;
|
||||
out.write(data, 0, bytesWritten);
|
||||
((Syncable) out).hflush();
|
||||
|
||||
|
@ -223,9 +219,9 @@ public abstract class CryptoStreamsTestBase {
|
|||
|
||||
private void verify(InputStream in, int bytesToVerify,
|
||||
byte[] expectedBytes) throws IOException {
|
||||
byte[] readBuf = new byte[bytesToVerify];
|
||||
final byte[] readBuf = new byte[bytesToVerify];
|
||||
readAll(in, readBuf, 0, bytesToVerify);
|
||||
for (int i=0; i<bytesToVerify; i++) {
|
||||
for (int i = 0; i < bytesToVerify; i++) {
|
||||
Assert.assertEquals(expectedBytes[i], readBuf[i]);
|
||||
}
|
||||
}
|
||||
|
@ -246,9 +242,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
return total;
|
||||
}
|
||||
|
||||
/**
|
||||
* Test positioned read.
|
||||
*/
|
||||
/** Test positioned read. */
|
||||
@Test(timeout=120000)
|
||||
public void testPositionedRead() throws Exception {
|
||||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
|
@ -256,10 +250,10 @@ public abstract class CryptoStreamsTestBase {
|
|||
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
// Pos: 1/3 dataLen
|
||||
positionedReadCheck(in , dataLen/3);
|
||||
positionedReadCheck(in , dataLen / 3);
|
||||
|
||||
// Pos: 1/2 dataLen
|
||||
positionedReadCheck(in, dataLen/2);
|
||||
positionedReadCheck(in, dataLen / 2);
|
||||
in.close();
|
||||
}
|
||||
|
||||
|
@ -275,25 +269,23 @@ public abstract class CryptoStreamsTestBase {
|
|||
Assert.assertArrayEquals(readData, expectedData);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test read fully
|
||||
*/
|
||||
/** Test read fully */
|
||||
@Test(timeout=120000)
|
||||
public void testReadFully() throws Exception {
|
||||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
writeData(out);
|
||||
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
final int len1 = dataLen/4;
|
||||
final int len1 = dataLen / 4;
|
||||
// Read len1 bytes
|
||||
byte [] readData = new byte[len1];
|
||||
byte[] readData = new byte[len1];
|
||||
readAll(in, readData, 0, len1);
|
||||
byte[] expectedData = new byte[len1];
|
||||
System.arraycopy(data, 0, expectedData, 0, len1);
|
||||
Assert.assertArrayEquals(readData, expectedData);
|
||||
|
||||
// Pos: 1/3 dataLen
|
||||
readFullyCheck(in, dataLen/3);
|
||||
readFullyCheck(in, dataLen / 3);
|
||||
|
||||
// Read len1 bytes
|
||||
readData = new byte[len1];
|
||||
|
@ -303,7 +295,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
Assert.assertArrayEquals(readData, expectedData);
|
||||
|
||||
// Pos: 1/2 dataLen
|
||||
readFullyCheck(in, dataLen/2);
|
||||
readFullyCheck(in, dataLen / 2);
|
||||
|
||||
// Read len1 bytes
|
||||
readData = new byte[len1];
|
||||
|
@ -331,9 +323,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test seek to different position.
|
||||
*/
|
||||
/** Test seek to different position. */
|
||||
@Test(timeout=120000)
|
||||
public void testSeek() throws Exception {
|
||||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
|
@ -341,13 +331,15 @@ public abstract class CryptoStreamsTestBase {
|
|||
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
// Pos: 1/3 dataLen
|
||||
seekCheck(in, dataLen/3);
|
||||
seekCheck(in, dataLen / 3);
|
||||
|
||||
// Pos: 0
|
||||
seekCheck(in, 0);
|
||||
|
||||
// Pos: 1/2 dataLen
|
||||
seekCheck(in, dataLen/2);
|
||||
seekCheck(in, dataLen / 2);
|
||||
|
||||
final long pos = ((Seekable) in).getPos();
|
||||
|
||||
// Pos: -3
|
||||
try {
|
||||
|
@ -355,8 +347,9 @@ public abstract class CryptoStreamsTestBase {
|
|||
Assert.fail("Seek to negative offset should fail.");
|
||||
} catch (IllegalArgumentException e) {
|
||||
GenericTestUtils.assertExceptionContains("Cannot seek to negative " +
|
||||
"offset", e);
|
||||
"offset", e);
|
||||
}
|
||||
Assert.assertEquals(pos, ((Seekable) in).getPos());
|
||||
|
||||
// Pos: dataLen + 3
|
||||
try {
|
||||
|
@ -365,6 +358,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
} catch (IOException e) {
|
||||
GenericTestUtils.assertExceptionContains("Cannot seek after EOF", e);
|
||||
}
|
||||
Assert.assertEquals(pos, ((Seekable) in).getPos());
|
||||
|
||||
in.close();
|
||||
}
|
||||
|
@ -382,9 +376,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
Assert.assertArrayEquals(readData, expectedData);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test get position.
|
||||
*/
|
||||
/** Test get position. */
|
||||
@Test(timeout=120000)
|
||||
public void testGetPos() throws Exception {
|
||||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
|
@ -393,7 +385,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
// Default buffer size
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
byte[] result = new byte[dataLen];
|
||||
int n1 = readAll(in, result, 0, dataLen/3);
|
||||
int n1 = readAll(in, result, 0, dataLen / 3);
|
||||
Assert.assertEquals(n1, ((Seekable) in).getPos());
|
||||
|
||||
int n2 = readAll(in, result, n1, dataLen - n1);
|
||||
|
@ -409,7 +401,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
// Default buffer size
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
byte[] result = new byte[dataLen];
|
||||
int n1 = readAll(in, result, 0, dataLen/3);
|
||||
int n1 = readAll(in, result, 0, dataLen / 3);
|
||||
Assert.assertEquals(in.available(), dataLen - n1);
|
||||
|
||||
int n2 = readAll(in, result, n1, dataLen - n1);
|
||||
|
@ -417,9 +409,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
in.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test skip.
|
||||
*/
|
||||
/** Test skip. */
|
||||
@Test(timeout=120000)
|
||||
public void testSkip() throws Exception {
|
||||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
|
@ -428,10 +418,10 @@ public abstract class CryptoStreamsTestBase {
|
|||
// Default buffer size
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
byte[] result = new byte[dataLen];
|
||||
int n1 = readAll(in, result, 0, dataLen/3);
|
||||
int n1 = readAll(in, result, 0, dataLen / 3);
|
||||
Assert.assertEquals(n1, ((Seekable) in).getPos());
|
||||
|
||||
long skipped = in.skip(dataLen/3);
|
||||
long skipped = in.skip(dataLen / 3);
|
||||
int n2 = readAll(in, result, 0, dataLen);
|
||||
|
||||
Assert.assertEquals(dataLen, n1 + skipped + n2);
|
||||
|
@ -468,9 +458,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
Assert.assertArrayEquals(readData, expectedData);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test byte buffer read with different buffer size.
|
||||
*/
|
||||
/** Test byte buffer read with different buffer size. */
|
||||
@Test(timeout=120000)
|
||||
public void testByteBufferRead() throws Exception {
|
||||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
|
@ -530,8 +518,8 @@ public abstract class CryptoStreamsTestBase {
|
|||
OutputStream out = getOutputStream(defaultBufferSize);
|
||||
writeData(out);
|
||||
|
||||
final int len1 = dataLen/8;
|
||||
final int len2 = dataLen/10;
|
||||
final int len1 = dataLen / 8;
|
||||
final int len2 = dataLen / 10;
|
||||
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
// Read len1 data.
|
||||
|
@ -551,7 +539,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
Assert.assertEquals(len2, n);
|
||||
|
||||
// Pos: 1/4 dataLen
|
||||
positionedReadCheck(in , dataLen/4);
|
||||
positionedReadCheck(in , dataLen / 4);
|
||||
|
||||
// Pos should be len1 + len2 + len2
|
||||
pos = ((Seekable) in).getPos();
|
||||
|
@ -572,7 +560,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
Assert.assertEquals(len1 + 2 * len2 + nRead, pos);
|
||||
|
||||
// Pos: 1/3 dataLen
|
||||
positionedReadCheck(in , dataLen/3);
|
||||
positionedReadCheck(in , dataLen / 3);
|
||||
|
||||
// Read forward len1
|
||||
readData = new byte[len1];
|
||||
|
@ -611,18 +599,18 @@ public abstract class CryptoStreamsTestBase {
|
|||
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
|
||||
final int len1 = dataLen/8;
|
||||
final int len1 = dataLen / 8;
|
||||
byte[] readData = new byte[len1];
|
||||
readAll(in, readData, 0, len1);
|
||||
|
||||
// Pos: 1/3 dataLen
|
||||
seekToNewSourceCheck(in, dataLen/3);
|
||||
seekToNewSourceCheck(in, dataLen / 3);
|
||||
|
||||
// Pos: 0
|
||||
seekToNewSourceCheck(in, 0);
|
||||
|
||||
// Pos: 1/2 dataLen
|
||||
seekToNewSourceCheck(in, dataLen/2);
|
||||
seekToNewSourceCheck(in, dataLen / 2);
|
||||
|
||||
// Pos: -3
|
||||
try {
|
||||
|
@ -638,7 +626,8 @@ public abstract class CryptoStreamsTestBase {
|
|||
seekToNewSourceCheck(in, dataLen + 3);
|
||||
Assert.fail("Seek after EOF should fail.");
|
||||
} catch (IOException e) {
|
||||
GenericTestUtils.assertExceptionContains("Attempted to read past end of file", e);
|
||||
GenericTestUtils.assertExceptionContains("Attempted to read past " +
|
||||
"end of file", e);
|
||||
}
|
||||
|
||||
in.close();
|
||||
|
@ -677,7 +666,7 @@ public abstract class CryptoStreamsTestBase {
|
|||
writeData(out);
|
||||
|
||||
InputStream in = getInputStream(defaultBufferSize);
|
||||
final int len1 = dataLen/8;
|
||||
final int len1 = dataLen / 8;
|
||||
// ByteBuffer size is len1
|
||||
ByteBuffer buffer = ((HasEnhancedByteBufferAccess) in).read(
|
||||
getBufferPool(), len1, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
|
||||
|
|
Loading…
Reference in New Issue