|
@@ -42,6 +42,10 @@ import org.apache.hadoop.io.DataInputBuffer;
|
|
|
import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
import org.junit.AfterClass;
|
|
|
import org.junit.BeforeClass;
|
|
|
+import org.junit.Test;
|
|
|
+
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
public class TestCryptoStreams extends CryptoStreamsTestBase {
|
|
|
/**
|
|
@@ -91,7 +95,7 @@ public class TestCryptoStreams extends CryptoStreamsTestBase {
|
|
|
}
|
|
|
|
|
|
private class FakeOutputStream extends OutputStream
|
|
|
- implements Syncable, CanSetDropBehind{
|
|
|
+ implements Syncable, CanSetDropBehind, StreamCapabilities{
|
|
|
private final byte[] oneByteBuf = new byte[1];
|
|
|
private final DataOutputBuffer out;
|
|
|
private boolean closed;
|
|
@@ -153,7 +157,19 @@ public class TestCryptoStreams extends CryptoStreamsTestBase {
|
|
|
checkStream();
|
|
|
flush();
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean hasCapability(String capability) {
|
|
|
+ switch (capability.toLowerCase()) {
|
|
|
+ case StreamCapabilities.HFLUSH:
|
|
|
+ case StreamCapabilities.HSYNC:
|
|
|
+ case StreamCapabilities.DROPBEHIND:
|
|
|
+ return true;
|
|
|
+ default:
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private void checkStream() throws IOException {
|
|
|
if (closed) {
|
|
|
throw new IOException("Stream is closed!");
|
|
@@ -393,4 +409,31 @@ public class TestCryptoStreams extends CryptoStreamsTestBase {
|
|
|
return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This tests {@link StreamCapabilities#hasCapability(String)} for the
|
|
|
+ * the underlying streams.
|
|
|
+ */
|
|
|
+ @Test(timeout = 120000)
|
|
|
+ public void testHasCapability() throws Exception {
|
|
|
+ // verify hasCapability returns what FakeOutputStream is set up for
|
|
|
+ CryptoOutputStream cos =
|
|
|
+ (CryptoOutputStream) getOutputStream(defaultBufferSize, key, iv);
|
|
|
+ assertTrue(cos instanceof StreamCapabilities);
|
|
|
+ assertTrue(cos.hasCapability(StreamCapabilities.HFLUSH));
|
|
|
+ assertTrue(cos.hasCapability(StreamCapabilities.HSYNC));
|
|
|
+ assertTrue(cos.hasCapability(StreamCapabilities.DROPBEHIND));
|
|
|
+ assertFalse(cos.hasCapability(StreamCapabilities.READAHEAD));
|
|
|
+ assertFalse(cos.hasCapability(StreamCapabilities.UNBUFFER));
|
|
|
+
|
|
|
+ // verify hasCapability for input stream
|
|
|
+ CryptoInputStream cis =
|
|
|
+ (CryptoInputStream) getInputStream(defaultBufferSize, key, iv);
|
|
|
+ assertTrue(cis instanceof StreamCapabilities);
|
|
|
+ assertTrue(cis.hasCapability(StreamCapabilities.DROPBEHIND));
|
|
|
+ assertTrue(cis.hasCapability(StreamCapabilities.READAHEAD));
|
|
|
+ assertTrue(cis.hasCapability(StreamCapabilities.UNBUFFER));
|
|
|
+ assertFalse(cis.hasCapability(StreamCapabilities.HFLUSH));
|
|
|
+ assertFalse(cis.hasCapability(StreamCapabilities.HSYNC));
|
|
|
+ }
|
|
|
}
|