You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ha...@apache.org on 2018/01/03 21:31:55 UTC

[23/50] [abbrv] hadoop git commit: HADOOP-15149. CryptoOutputStream should implement StreamCapabilities.

HADOOP-15149. CryptoOutputStream should implement StreamCapabilities.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/81127616
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/81127616
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/81127616

Branch: refs/heads/YARN-1011
Commit: 81127616c571b7cd25686e60a1105f96ca0626b7
Parents: b82049b
Author: Xiao Chen <xi...@apache.org>
Authored: Fri Dec 29 13:40:42 2017 -0800
Committer: Xiao Chen <xi...@apache.org>
Committed: Fri Dec 29 13:41:15 2017 -0800

----------------------------------------------------------------------
 .../hadoop/crypto/CryptoOutputStream.java       | 11 ++++-
 .../hadoop/crypto/CryptoStreamsTestBase.java    |  4 +-
 .../apache/hadoop/crypto/TestCryptoStreams.java | 47 +++++++++++++++++++-
 3 files changed, 57 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81127616/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
index 9fb0ff6..2f347c5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java
@@ -26,6 +26,7 @@ import java.security.GeneralSecurityException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.CanSetDropBehind;
+import org.apache.hadoop.fs.StreamCapabilities;
 import org.apache.hadoop.fs.Syncable;
 
 import com.google.common.base.Preconditions;
@@ -47,7 +48,7 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 public class CryptoOutputStream extends FilterOutputStream implements 
-    Syncable, CanSetDropBehind {
+    Syncable, CanSetDropBehind, StreamCapabilities {
   private final byte[] oneByteBuf = new byte[1];
   private final CryptoCodec codec;
   private final Encryptor encryptor;
@@ -304,4 +305,12 @@ public class CryptoOutputStream extends FilterOutputStream implements
     CryptoStreamUtils.freeDB(inBuffer);
     CryptoStreamUtils.freeDB(outBuffer);
   }
+
+  @Override
+  public boolean hasCapability(String capability) {
+    if (out instanceof StreamCapabilities) {
+      return ((StreamCapabilities) out).hasCapability(capability);
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81127616/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java
index 259383d..a0eb105 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java
@@ -50,9 +50,9 @@ public abstract class CryptoStreamsTestBase {
       CryptoStreamsTestBase.class);
 
   protected static CryptoCodec codec;
-  private static final byte[] key = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 
+  protected static final byte[] key = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06,
     0x07, 0x08, 0x09, 0x10, 0x11, 0x12, 0x13, 0x14, 0x15, 0x16};
-  private static final byte[] iv = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 
+  protected static final byte[] iv = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06,
     0x07, 0x08, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08};
   
   protected static final int count = 10000;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81127616/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java
index 027ac93..2172d8a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreams.java
@@ -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));
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org