|
@@ -30,20 +30,23 @@ import java.util.EnumSet;
|
|
import java.util.Queue;
|
|
import java.util.Queue;
|
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
|
import java.util.concurrent.ConcurrentLinkedQueue;
|
|
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
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.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;
|
|
|
|
+import org.apache.hadoop.fs.CanUnbuffer;
|
|
import org.apache.hadoop.fs.FSExceptionMessages;
|
|
import org.apache.hadoop.fs.FSExceptionMessages;
|
|
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
|
|
import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
|
|
import org.apache.hadoop.fs.HasFileDescriptor;
|
|
import org.apache.hadoop.fs.HasFileDescriptor;
|
|
import org.apache.hadoop.fs.PositionedReadable;
|
|
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.fs.StreamCapabilities;
|
|
|
|
+import org.apache.hadoop.fs.StreamCapabilitiesPolicy;
|
|
import org.apache.hadoop.io.ByteBufferPool;
|
|
import org.apache.hadoop.io.ByteBufferPool;
|
|
-
|
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
|
|
|
/**
|
|
/**
|
|
* CryptoInputStream decrypts data. It is not thread-safe. AES CTR mode is
|
|
* CryptoInputStream decrypts data. It is not thread-safe. AES CTR mode is
|
|
@@ -61,7 +64,7 @@ import com.google.common.base.Preconditions;
|
|
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,
|
|
- ReadableByteChannel {
|
|
|
|
|
|
+ ReadableByteChannel, CanUnbuffer, StreamCapabilities {
|
|
private final byte[] oneByteBuf = new byte[1];
|
|
private final byte[] oneByteBuf = new byte[1];
|
|
private final CryptoCodec codec;
|
|
private final CryptoCodec codec;
|
|
private final Decryptor decryptor;
|
|
private final Decryptor decryptor;
|
|
@@ -719,4 +722,27 @@ public class CryptoInputStream extends FilterInputStream implements
|
|
public boolean isOpen() {
|
|
public boolean isOpen() {
|
|
return !closed;
|
|
return !closed;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ private void cleanDecryptorPool() {
|
|
|
|
+ decryptorPool.clear();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void unbuffer() {
|
|
|
|
+ cleanBufferPool();
|
|
|
|
+ cleanDecryptorPool();
|
|
|
|
+ StreamCapabilitiesPolicy.unbuffer(in);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public boolean hasCapability(String capability) {
|
|
|
|
+ switch (StringUtils.toLowerCase(capability)) {
|
|
|
|
+ case StreamCapabilities.READAHEAD:
|
|
|
|
+ case StreamCapabilities.DROPBEHIND:
|
|
|
|
+ case StreamCapabilities.UNBUFFER:
|
|
|
|
+ return true;
|
|
|
|
+ default:
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|