|
@@ -25,10 +25,11 @@ import java.io.InputStream;
|
|
import java.nio.ByteBuffer;
|
|
import java.nio.ByteBuffer;
|
|
import java.security.GeneralSecurityException;
|
|
import java.security.GeneralSecurityException;
|
|
import java.util.EnumSet;
|
|
import java.util.EnumSet;
|
|
|
|
+import java.util.Queue;
|
|
|
|
+import java.util.concurrent.ConcurrentLinkedQueue;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
-import org.apache.hadoop.conf.Configuration;
|
|
|
|
import org.apache.hadoop.fs.ByteBufferReadable;
|
|
import org.apache.hadoop.fs.ByteBufferReadable;
|
|
import org.apache.hadoop.fs.CanSetDropBehind;
|
|
import org.apache.hadoop.fs.CanSetDropBehind;
|
|
import org.apache.hadoop.fs.CanSetReadahead;
|
|
import org.apache.hadoop.fs.CanSetReadahead;
|
|
@@ -38,8 +39,6 @@ import org.apache.hadoop.fs.PositionedReadable;
|
|
import org.apache.hadoop.fs.ReadOption;
|
|
import org.apache.hadoop.fs.ReadOption;
|
|
import org.apache.hadoop.fs.Seekable;
|
|
import org.apache.hadoop.fs.Seekable;
|
|
import org.apache.hadoop.io.ByteBufferPool;
|
|
import org.apache.hadoop.io.ByteBufferPool;
|
|
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_KEY;
|
|
|
|
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_DEFAULT;
|
|
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
|
|
|
|
@@ -54,15 +53,15 @@ import com.google.common.base.Preconditions;
|
|
* <p/>
|
|
* <p/>
|
|
* The underlying stream offset is maintained as state.
|
|
* The underlying stream offset is maintained as state.
|
|
*/
|
|
*/
|
|
-@InterfaceAudience.Public
|
|
|
|
|
|
+@InterfaceAudience.Private
|
|
@InterfaceStability.Evolving
|
|
@InterfaceStability.Evolving
|
|
public class CryptoInputStream extends FilterInputStream implements
|
|
public class CryptoInputStream extends FilterInputStream implements
|
|
Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor,
|
|
Seekable, PositionedReadable, ByteBufferReadable, HasFileDescriptor,
|
|
CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess {
|
|
CanSetDropBehind, CanSetReadahead, HasEnhancedByteBufferAccess {
|
|
- private static final int MIN_BUFFER_SIZE = 512;
|
|
|
|
private static final byte[] oneByteBuf = new byte[1];
|
|
private static final byte[] oneByteBuf = new byte[1];
|
|
private final CryptoCodec codec;
|
|
private final CryptoCodec codec;
|
|
private final Decryptor decryptor;
|
|
private final Decryptor decryptor;
|
|
|
|
+ private final int bufferSize;
|
|
|
|
|
|
/**
|
|
/**
|
|
* Input data buffer. The data starts at inBuffer.position() and ends at
|
|
* Input data buffer. The data starts at inBuffer.position() and ends at
|
|
@@ -79,7 +78,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
|
|
|
|
/**
|
|
/**
|
|
* Whether the underlying stream supports
|
|
* Whether the underlying stream supports
|
|
- * {@link #org.apache.hadoop.fs.ByteBufferReadable}
|
|
|
|
|
|
+ * {@link org.apache.hadoop.fs.ByteBufferReadable}
|
|
*/
|
|
*/
|
|
private Boolean usingByteBufferRead = null;
|
|
private Boolean usingByteBufferRead = null;
|
|
|
|
|
|
@@ -94,32 +93,33 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
private final byte[] initIV;
|
|
private final byte[] initIV;
|
|
private byte[] iv;
|
|
private byte[] iv;
|
|
|
|
|
|
|
|
+ /** DirectBuffer pool */
|
|
|
|
+ private final Queue<ByteBuffer> bufferPool =
|
|
|
|
+ new ConcurrentLinkedQueue<ByteBuffer>();
|
|
|
|
+ /** Decryptor pool */
|
|
|
|
+ private final Queue<Decryptor> decryptorPool =
|
|
|
|
+ new ConcurrentLinkedQueue<Decryptor>();
|
|
|
|
+
|
|
public CryptoInputStream(InputStream in, CryptoCodec codec,
|
|
public CryptoInputStream(InputStream in, CryptoCodec codec,
|
|
int bufferSize, byte[] key, byte[] iv) throws IOException {
|
|
int bufferSize, byte[] key, byte[] iv) throws IOException {
|
|
super(in);
|
|
super(in);
|
|
- Preconditions.checkArgument(bufferSize >= MIN_BUFFER_SIZE,
|
|
|
|
- "Minimum value of buffer size is 512.");
|
|
|
|
- this.key = key;
|
|
|
|
- this.initIV = iv;
|
|
|
|
- this.iv = iv.clone();
|
|
|
|
- inBuffer = ByteBuffer.allocateDirect(bufferSize);
|
|
|
|
- outBuffer = ByteBuffer.allocateDirect(bufferSize);
|
|
|
|
- outBuffer.limit(0);
|
|
|
|
|
|
+ this.bufferSize = CryptoStreamUtils.checkBufferSize(codec, bufferSize);
|
|
this.codec = codec;
|
|
this.codec = codec;
|
|
- try {
|
|
|
|
- decryptor = codec.getDecryptor();
|
|
|
|
- } catch (GeneralSecurityException e) {
|
|
|
|
- throw new IOException(e);
|
|
|
|
- }
|
|
|
|
|
|
+ this.key = key.clone();
|
|
|
|
+ this.initIV = iv.clone();
|
|
|
|
+ this.iv = iv.clone();
|
|
|
|
+ inBuffer = ByteBuffer.allocateDirect(this.bufferSize);
|
|
|
|
+ outBuffer = ByteBuffer.allocateDirect(this.bufferSize);
|
|
|
|
+ decryptor = getDecryptor();
|
|
if (in instanceof Seekable) {
|
|
if (in instanceof Seekable) {
|
|
streamOffset = ((Seekable) in).getPos();
|
|
streamOffset = ((Seekable) in).getPos();
|
|
}
|
|
}
|
|
- updateDecryptor();
|
|
|
|
|
|
+ resetStreamOffset(streamOffset);
|
|
}
|
|
}
|
|
|
|
|
|
public CryptoInputStream(InputStream in, CryptoCodec codec,
|
|
public CryptoInputStream(InputStream in, CryptoCodec codec,
|
|
byte[] key, byte[] iv) throws IOException {
|
|
byte[] key, byte[] iv) throws IOException {
|
|
- this(in, codec, getBufferSize(codec.getConf()), key, iv);
|
|
|
|
|
|
+ this(in, codec, CryptoStreamUtils.getBufferSize(codec.getConf()), key, iv);
|
|
}
|
|
}
|
|
|
|
|
|
public InputStream getWrappedStream() {
|
|
public InputStream getWrappedStream() {
|
|
@@ -169,14 +169,14 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
usingByteBufferRead = Boolean.FALSE;
|
|
usingByteBufferRead = Boolean.FALSE;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- if (!usingByteBufferRead.booleanValue()) {
|
|
|
|
- n = readFromUnderlyingStream();
|
|
|
|
|
|
+ if (!usingByteBufferRead) {
|
|
|
|
+ n = readFromUnderlyingStream(inBuffer);
|
|
}
|
|
}
|
|
} else {
|
|
} else {
|
|
- if (usingByteBufferRead.booleanValue()) {
|
|
|
|
|
|
+ if (usingByteBufferRead) {
|
|
n = ((ByteBufferReadable) in).read(inBuffer);
|
|
n = ((ByteBufferReadable) in).read(inBuffer);
|
|
} else {
|
|
} else {
|
|
- n = readFromUnderlyingStream();
|
|
|
|
|
|
+ n = readFromUnderlyingStream(inBuffer);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (n <= 0) {
|
|
if (n <= 0) {
|
|
@@ -184,7 +184,8 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
}
|
|
}
|
|
|
|
|
|
streamOffset += n; // Read n bytes
|
|
streamOffset += n; // Read n bytes
|
|
- decrypt();
|
|
|
|
|
|
+ decrypt(decryptor, inBuffer, outBuffer, padding);
|
|
|
|
+ padding = afterDecryption(decryptor, inBuffer, streamOffset, iv);
|
|
n = Math.min(len, outBuffer.remaining());
|
|
n = Math.min(len, outBuffer.remaining());
|
|
outBuffer.get(b, off, n);
|
|
outBuffer.get(b, off, n);
|
|
return n;
|
|
return n;
|
|
@@ -192,7 +193,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
}
|
|
}
|
|
|
|
|
|
/** Read data from underlying stream. */
|
|
/** Read data from underlying stream. */
|
|
- private int readFromUnderlyingStream() throws IOException {
|
|
|
|
|
|
+ private int readFromUnderlyingStream(ByteBuffer inBuffer) throws IOException {
|
|
final int toRead = inBuffer.remaining();
|
|
final int toRead = inBuffer.remaining();
|
|
final byte[] tmp = getTmpBuf();
|
|
final byte[] tmp = getTmpBuf();
|
|
final int n = in.read(tmp, 0, toRead);
|
|
final int n = in.read(tmp, 0, toRead);
|
|
@@ -205,16 +206,18 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
private byte[] tmpBuf;
|
|
private byte[] tmpBuf;
|
|
private byte[] getTmpBuf() {
|
|
private byte[] getTmpBuf() {
|
|
if (tmpBuf == null) {
|
|
if (tmpBuf == null) {
|
|
- tmpBuf = new byte[inBuffer.capacity()];
|
|
|
|
|
|
+ tmpBuf = new byte[bufferSize];
|
|
}
|
|
}
|
|
return tmpBuf;
|
|
return tmpBuf;
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Do the decryption using {@link #inBuffer} as input and {@link #outBuffer}
|
|
|
|
- * as output.
|
|
|
|
|
|
+ * Do the decryption using inBuffer as input and outBuffer as output.
|
|
|
|
+ * Upon return, inBuffer is cleared; the decrypted data starts at
|
|
|
|
+ * outBuffer.position() and ends at outBuffer.limit();
|
|
*/
|
|
*/
|
|
- private void decrypt() throws IOException {
|
|
|
|
|
|
+ private void decrypt(Decryptor decryptor, ByteBuffer inBuffer,
|
|
|
|
+ ByteBuffer outBuffer, byte padding) throws IOException {
|
|
Preconditions.checkState(inBuffer.position() >= padding);
|
|
Preconditions.checkState(inBuffer.position() >= padding);
|
|
if(inBuffer.position() == padding) {
|
|
if(inBuffer.position() == padding) {
|
|
// There is no real data in inBuffer.
|
|
// There is no real data in inBuffer.
|
|
@@ -231,8 +234,16 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
* same position.
|
|
* same position.
|
|
*/
|
|
*/
|
|
outBuffer.position(padding);
|
|
outBuffer.position(padding);
|
|
- padding = 0;
|
|
|
|
}
|
|
}
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * This method is executed immediately after decryption. Check whether
|
|
|
|
+ * decryptor should be updated and recalculate padding if needed.
|
|
|
|
+ */
|
|
|
|
+ private byte afterDecryption(Decryptor decryptor, ByteBuffer inBuffer,
|
|
|
|
+ long position, byte[] iv) throws IOException {
|
|
|
|
+ byte padding = 0;
|
|
if (decryptor.isContextReset()) {
|
|
if (decryptor.isContextReset()) {
|
|
/*
|
|
/*
|
|
* This code is generally not executed since the decryptor usually
|
|
* This code is generally not executed since the decryptor usually
|
|
@@ -240,23 +251,31 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
* some implementations can't maintain context so a re-init is necessary
|
|
* some implementations can't maintain context so a re-init is necessary
|
|
* after each decryption call.
|
|
* after each decryption call.
|
|
*/
|
|
*/
|
|
- updateDecryptor();
|
|
|
|
|
|
+ updateDecryptor(decryptor, position, iv);
|
|
|
|
+ padding = getPadding(position);
|
|
|
|
+ inBuffer.position(padding);
|
|
}
|
|
}
|
|
|
|
+ return padding;
|
|
}
|
|
}
|
|
|
|
|
|
- /**
|
|
|
|
- * Update the {@link #decryptor}. Calculate the counter and {@link #padding}.
|
|
|
|
- */
|
|
|
|
- private void updateDecryptor() throws IOException {
|
|
|
|
- final long counter = streamOffset / codec.getAlgorithmBlockSize();
|
|
|
|
- padding = (byte)(streamOffset % codec.getAlgorithmBlockSize());
|
|
|
|
- inBuffer.position(padding); // Set proper position for input data.
|
|
|
|
|
|
+ private long getCounter(long position) {
|
|
|
|
+ return position / codec.getAlgorithmBlockSize();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private byte getPadding(long position) {
|
|
|
|
+ return (byte)(position % codec.getAlgorithmBlockSize());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Calculate the counter and iv, update the decryptor. */
|
|
|
|
+ private void updateDecryptor(Decryptor decryptor, long position, byte[] iv)
|
|
|
|
+ throws IOException {
|
|
|
|
+ final long counter = getCounter(position);
|
|
codec.calculateIV(initIV, counter, iv);
|
|
codec.calculateIV(initIV, counter, iv);
|
|
decryptor.init(key, iv);
|
|
decryptor.init(key, iv);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Reset the underlying stream offset, and clear {@link #inBuffer} and
|
|
|
|
|
|
+ * Reset the underlying stream offset; clear {@link #inBuffer} and
|
|
* {@link #outBuffer}. This Typically happens during {@link #seek(long)}
|
|
* {@link #outBuffer}. This Typically happens during {@link #seek(long)}
|
|
* or {@link #skip(long)}.
|
|
* or {@link #skip(long)}.
|
|
*/
|
|
*/
|
|
@@ -265,7 +284,9 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
inBuffer.clear();
|
|
inBuffer.clear();
|
|
outBuffer.clear();
|
|
outBuffer.clear();
|
|
outBuffer.limit(0);
|
|
outBuffer.limit(0);
|
|
- updateDecryptor();
|
|
|
|
|
|
+ updateDecryptor(decryptor, offset, iv);
|
|
|
|
+ padding = getPadding(offset);
|
|
|
|
+ inBuffer.position(padding); // Set proper position for input data.
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -279,17 +300,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
closed = true;
|
|
closed = true;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Forcibly free the direct buffer. */
|
|
|
|
- private void freeBuffers() {
|
|
|
|
- final sun.misc.Cleaner inBufferCleaner =
|
|
|
|
- ((sun.nio.ch.DirectBuffer) inBuffer).cleaner();
|
|
|
|
- inBufferCleaner.clean();
|
|
|
|
- final sun.misc.Cleaner outBufferCleaner =
|
|
|
|
- ((sun.nio.ch.DirectBuffer) outBuffer).cleaner();
|
|
|
|
- outBufferCleaner.clean();
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- /** Positioned read. */
|
|
|
|
|
|
+ /** Positioned read. It is thread-safe */
|
|
@Override
|
|
@Override
|
|
public int read(long position, byte[] buffer, int offset, int length)
|
|
public int read(long position, byte[] buffer, int offset, int length)
|
|
throws IOException {
|
|
throws IOException {
|
|
@@ -298,12 +309,8 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
final int n = ((PositionedReadable) in).read(position, buffer, offset,
|
|
final int n = ((PositionedReadable) in).read(position, buffer, offset,
|
|
length);
|
|
length);
|
|
if (n > 0) {
|
|
if (n > 0) {
|
|
- /*
|
|
|
|
- * Since this operation does not change the current offset of a file,
|
|
|
|
- * streamOffset should not be changed. We need to restore the decryptor
|
|
|
|
- * and outBuffer after decryption.
|
|
|
|
- */
|
|
|
|
- decrypt(position, buffer, offset, length);
|
|
|
|
|
|
+ // This operation does not change the current offset of the file
|
|
|
|
+ decrypt(position, buffer, offset, n);
|
|
}
|
|
}
|
|
|
|
|
|
return n;
|
|
return n;
|
|
@@ -315,39 +322,39 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
|
|
|
|
/**
|
|
/**
|
|
* Decrypt length bytes in buffer starting at offset. Output is also put
|
|
* 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.
|
|
|
|
|
|
+ * into buffer starting at offset. It is thread-safe.
|
|
*/
|
|
*/
|
|
private void decrypt(long position, byte[] buffer, int offset, int length)
|
|
private void decrypt(long position, byte[] buffer, int offset, int length)
|
|
throws IOException {
|
|
throws IOException {
|
|
- final byte[] tmp = getTmpBuf();
|
|
|
|
- int unread = outBuffer.remaining();
|
|
|
|
- if (unread > 0) { // Cache outBuffer
|
|
|
|
- outBuffer.get(tmp, 0, unread);
|
|
|
|
- }
|
|
|
|
- final long curOffset = streamOffset;
|
|
|
|
- resetStreamOffset(position);
|
|
|
|
-
|
|
|
|
- int n = 0;
|
|
|
|
- while (n < length) {
|
|
|
|
- final int toDecrypt = Math.min(length - n, inBuffer.remaining());
|
|
|
|
- inBuffer.put(buffer, offset + n, toDecrypt);
|
|
|
|
- // Do decryption
|
|
|
|
- decrypt();
|
|
|
|
- outBuffer.get(buffer, offset + n, toDecrypt);
|
|
|
|
- n += toDecrypt;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- // After decryption
|
|
|
|
- resetStreamOffset(curOffset);
|
|
|
|
- if (unread > 0) { // Restore outBuffer
|
|
|
|
- outBuffer.clear();
|
|
|
|
- outBuffer.put(tmp, 0, unread);
|
|
|
|
- outBuffer.flip();
|
|
|
|
|
|
+ ByteBuffer inBuffer = getBuffer();
|
|
|
|
+ ByteBuffer outBuffer = getBuffer();
|
|
|
|
+ Decryptor decryptor = null;
|
|
|
|
+ try {
|
|
|
|
+ decryptor = getDecryptor();
|
|
|
|
+ byte[] iv = initIV.clone();
|
|
|
|
+ updateDecryptor(decryptor, position, iv);
|
|
|
|
+ byte padding = getPadding(position);
|
|
|
|
+ inBuffer.position(padding); // Set proper position for input data.
|
|
|
|
+
|
|
|
|
+ int n = 0;
|
|
|
|
+ while (n < length) {
|
|
|
|
+ int toDecrypt = Math.min(length - n, inBuffer.remaining());
|
|
|
|
+ inBuffer.put(buffer, offset + n, toDecrypt);
|
|
|
|
+ // Do decryption
|
|
|
|
+ decrypt(decryptor, inBuffer, outBuffer, padding);
|
|
|
|
+
|
|
|
|
+ outBuffer.get(buffer, offset + n, toDecrypt);
|
|
|
|
+ n += toDecrypt;
|
|
|
|
+ padding = afterDecryption(decryptor, inBuffer, position + n, iv);
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ returnBuffer(inBuffer);
|
|
|
|
+ returnBuffer(outBuffer);
|
|
|
|
+ returnDecryptor(decryptor);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- /** Positioned read fully. */
|
|
|
|
|
|
+ /** Positioned read fully. It is thread-safe */
|
|
@Override
|
|
@Override
|
|
public void readFully(long position, byte[] buffer, int offset, int length)
|
|
public void readFully(long position, byte[] buffer, int offset, int length)
|
|
throws IOException {
|
|
throws IOException {
|
|
@@ -355,11 +362,7 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
try {
|
|
try {
|
|
((PositionedReadable) in).readFully(position, buffer, offset, length);
|
|
((PositionedReadable) in).readFully(position, buffer, offset, length);
|
|
if (length > 0) {
|
|
if (length > 0) {
|
|
- /*
|
|
|
|
- * 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.
|
|
|
|
- */
|
|
|
|
|
|
+ // This operation does not change the current offset of the file
|
|
decrypt(position, buffer, offset, length);
|
|
decrypt(position, buffer, offset, length);
|
|
}
|
|
}
|
|
} catch (ClassCastException e) {
|
|
} catch (ClassCastException e) {
|
|
@@ -484,12 +487,15 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
buf.limit(start + len + Math.min(n - len, inBuffer.remaining()));
|
|
buf.limit(start + len + Math.min(n - len, inBuffer.remaining()));
|
|
inBuffer.put(buf);
|
|
inBuffer.put(buf);
|
|
// Do decryption
|
|
// Do decryption
|
|
- decrypt();
|
|
|
|
-
|
|
|
|
- buf.position(start + len);
|
|
|
|
- buf.limit(limit);
|
|
|
|
- len += outBuffer.remaining();
|
|
|
|
- buf.put(outBuffer);
|
|
|
|
|
|
+ try {
|
|
|
|
+ decrypt(decryptor, inBuffer, outBuffer, padding);
|
|
|
|
+ buf.position(start + len);
|
|
|
|
+ buf.limit(limit);
|
|
|
|
+ len += outBuffer.remaining();
|
|
|
|
+ buf.put(outBuffer);
|
|
|
|
+ } finally {
|
|
|
|
+ padding = afterDecryption(decryptor, inBuffer, streamOffset - (n - len), iv);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
buf.position(pos);
|
|
buf.position(pos);
|
|
}
|
|
}
|
|
@@ -612,8 +618,57 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- private static int getBufferSize(Configuration conf) {
|
|
|
|
- return conf.getInt(HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_KEY,
|
|
|
|
- HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_DEFAULT);
|
|
|
|
|
|
+ /** Get direct buffer from pool */
|
|
|
|
+ private ByteBuffer getBuffer() {
|
|
|
|
+ ByteBuffer buffer = bufferPool.poll();
|
|
|
|
+ if (buffer == null) {
|
|
|
|
+ buffer = ByteBuffer.allocateDirect(bufferSize);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return buffer;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Return direct buffer to pool */
|
|
|
|
+ private void returnBuffer(ByteBuffer buf) {
|
|
|
|
+ if (buf != null) {
|
|
|
|
+ buf.clear();
|
|
|
|
+ bufferPool.add(buf);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Forcibly free the direct buffers. */
|
|
|
|
+ private void freeBuffers() {
|
|
|
|
+ CryptoStreamUtils.freeDB(inBuffer);
|
|
|
|
+ CryptoStreamUtils.freeDB(outBuffer);
|
|
|
|
+ cleanBufferPool();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Clean direct buffer pool */
|
|
|
|
+ private void cleanBufferPool() {
|
|
|
|
+ ByteBuffer buf;
|
|
|
|
+ while ((buf = bufferPool.poll()) != null) {
|
|
|
|
+ CryptoStreamUtils.freeDB(buf);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Get decryptor from pool */
|
|
|
|
+ private Decryptor getDecryptor() throws IOException {
|
|
|
|
+ Decryptor decryptor = decryptorPool.poll();
|
|
|
|
+ if (decryptor == null) {
|
|
|
|
+ try {
|
|
|
|
+ decryptor = codec.createDecryptor();
|
|
|
|
+ } catch (GeneralSecurityException e) {
|
|
|
|
+ throw new IOException(e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return decryptor;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Return decryptor to pool */
|
|
|
|
+ private void returnDecryptor(Decryptor decryptor) {
|
|
|
|
+ if (decryptor != null) {
|
|
|
|
+ decryptorPool.add(decryptor);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|