Bläddra i källkod

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
Charles Lamb 11 år sedan
förälder
incheckning
2e5ae1aad7

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES-fs-encryption.txt

@@ -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

+ 3 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AESCTRCryptoCodec.java

@@ -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);

+ 14 - 14
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java

@@ -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

+ 65 - 59
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java

@@ -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);
         }
       }

+ 21 - 20
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java

@@ -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 {

+ 19 - 22
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/Decryptor.java

@@ -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) 

+ 19 - 23
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/Encryptor.java

@@ -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) 

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JCEAESCTRDecryptor.java

@@ -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)

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JCEAESCTREncryptor.java

@@ -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)

+ 1 - 2
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -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>

+ 45 - 56
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/CryptoStreamsTestBase.java

@@ -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));