소스 검색

HADOOP-10693: add files left out of previous checkin (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1607769 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 년 전
부모
커밋
03c858dad4

+ 71 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/AesCtrCryptoCodec.java

@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import com.google.common.base.Preconditions;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public abstract class AesCtrCryptoCodec extends CryptoCodec {
+
+  protected static final CipherSuite SUITE = CipherSuite.AES_CTR_NOPADDING;
+
+  /**
+   * For AES, the algorithm block is fixed size of 128 bits.
+   * @see http://en.wikipedia.org/wiki/Advanced_Encryption_Standard
+   */
+  private static final int AES_BLOCK_SIZE = SUITE.getAlgorithmBlockSize();
+  private static final int CTR_OFFSET = 8;
+
+  @Override
+  public CipherSuite getCipherSuite() {
+    return SUITE;
+  }
+  
+  /**
+   * 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);
+    
+    System.arraycopy(initIV, 0, IV, 0, CTR_OFFSET);
+    long l = (initIV[CTR_OFFSET + 0] << 56)
+        + ((initIV[CTR_OFFSET + 1] & 0xFF) << 48)
+        + ((initIV[CTR_OFFSET + 2] & 0xFF) << 40)
+        + ((initIV[CTR_OFFSET + 3] & 0xFF) << 32)
+        + ((initIV[CTR_OFFSET + 4] & 0xFF) << 24)
+        + ((initIV[CTR_OFFSET + 5] & 0xFF) << 16)
+        + ((initIV[CTR_OFFSET + 6] & 0xFF) << 8)
+        + (initIV[CTR_OFFSET + 7] & 0xFF);
+    l += counter;
+    IV[CTR_OFFSET + 0] = (byte) (l >>> 56);
+    IV[CTR_OFFSET + 1] = (byte) (l >>> 48);
+    IV[CTR_OFFSET + 2] = (byte) (l >>> 40);
+    IV[CTR_OFFSET + 3] = (byte) (l >>> 32);
+    IV[CTR_OFFSET + 4] = (byte) (l >>> 24);
+    IV[CTR_OFFSET + 5] = (byte) (l >>> 16);
+    IV[CTR_OFFSET + 6] = (byte) (l >>> 8);
+    IV[CTR_OFFSET + 7] = (byte) (l);
+  }
+}

+ 159 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/JceAesCtrCryptoCodec.java

@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.GeneralSecurityException;
+import java.security.SecureRandom;
+
+import javax.crypto.Cipher;
+import javax.crypto.spec.IvParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Preconditions;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_SECURE_RANDOM_ALGORITHM_KEY;
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_SECURE_RANDOM_ALGORITHM_DEFAULT;
+
+/**
+ * Implement the AES-CTR crypto codec using JCE provider.
+ */
+@InterfaceAudience.Private
+public class JceAesCtrCryptoCodec extends AesCtrCryptoCodec {
+  private Configuration conf;
+  private String provider;
+  private SecureRandom random;
+
+  public JceAesCtrCryptoCodec() {
+  }
+  
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+  
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    provider = conf.get(HADOOP_SECURITY_CRYPTO_JCE_PROVIDER_KEY);
+    final String secureRandomAlg = conf.get(
+        HADOOP_SECURITY_SECURE_RANDOM_ALGORITHM_KEY, 
+        HADOOP_SECURITY_SECURE_RANDOM_ALGORITHM_DEFAULT);
+    try {
+      random = (provider != null) ? 
+          SecureRandom.getInstance(secureRandomAlg, provider) : 
+            SecureRandom.getInstance(secureRandomAlg);
+    } catch (GeneralSecurityException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  @Override
+  public Encryptor createEncryptor() throws GeneralSecurityException {
+    return new JceAesCtrCipher(Cipher.ENCRYPT_MODE, provider);
+  }
+
+  @Override
+  public Decryptor createDecryptor() throws GeneralSecurityException {
+    return new JceAesCtrCipher(Cipher.DECRYPT_MODE, provider);
+  }
+  
+  @Override
+  public void generateSecureRandom(byte[] bytes) {
+    random.nextBytes(bytes);
+  }  
+  
+  private static class JceAesCtrCipher implements Encryptor, Decryptor {
+    private final Cipher cipher;
+    private final int mode;
+    private boolean contextReset = false;
+    
+    public JceAesCtrCipher(int mode, String provider) 
+        throws GeneralSecurityException {
+      this.mode = mode;
+      if (provider == null || provider.isEmpty()) {
+        cipher = Cipher.getInstance(SUITE.getName());
+      } else {
+        cipher = Cipher.getInstance(SUITE.getName(), provider);
+      }
+    }
+
+    @Override
+    public void init(byte[] key, byte[] iv) throws IOException {
+      Preconditions.checkNotNull(key);
+      Preconditions.checkNotNull(iv);
+      contextReset = false;
+      try {
+        cipher.init(mode, new SecretKeySpec(key, "AES"), 
+            new IvParameterSpec(iv));
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+
+    /**
+     * 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)
+        throws IOException {
+      process(inBuffer, outBuffer);
+    }
+    
+    /**
+     * 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)
+        throws IOException {
+      process(inBuffer, outBuffer);
+    }
+    
+    private void process(ByteBuffer inBuffer, ByteBuffer outBuffer)
+        throws IOException {
+      try {
+        int inputSize = inBuffer.remaining();
+        // Cipher#update will maintain crypto context.
+        int n = cipher.update(inBuffer, outBuffer);
+        if (n < inputSize) {
+          /**
+           * Typically code will not get here. Cipher#update will consume all 
+           * input data and put result in outBuffer. 
+           * Cipher#doFinal will reset the crypto context.
+           */
+          contextReset = true;
+          cipher.doFinal(inBuffer, outBuffer);
+        }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+    
+    @Override
+    public boolean isContextReset() {
+      return contextReset;
+    }
+  }
+}

+ 130 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslAesCtrCryptoCodec.java

@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.GeneralSecurityException;
+import java.security.SecureRandom;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Implement the AES-CTR crypto codec using JNI into OpenSSL.
+ */
+@InterfaceAudience.Private
+public class OpensslAesCtrCryptoCodec extends AesCtrCryptoCodec {
+  private Configuration conf;
+  private SecureRandom random = new SecureRandom();
+  
+  public OpensslAesCtrCryptoCodec() {
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+
+  @Override
+  public Encryptor createEncryptor() throws GeneralSecurityException {
+    return new OpensslAesCtrCipher(OpensslCipher.ENCRYPT_MODE);
+  }
+
+  @Override
+  public Decryptor createDecryptor() throws GeneralSecurityException {
+    return new OpensslAesCtrCipher(OpensslCipher.DECRYPT_MODE);
+  }
+  
+  @Override
+  public void generateSecureRandom(byte[] bytes) {
+    random.nextBytes(bytes);
+  }
+  
+  private static class OpensslAesCtrCipher implements Encryptor, Decryptor {
+    private final OpensslCipher cipher;
+    private final int mode;
+    private boolean contextReset = false;
+    
+    public OpensslAesCtrCipher(int mode) throws GeneralSecurityException {
+      this.mode = mode;
+      cipher = OpensslCipher.getInstance(OpensslCipher.AES_CTR, 
+          OpensslCipher.PADDING_NOPADDING);
+    }
+
+    @Override
+    public void init(byte[] key, byte[] iv) throws IOException {
+      Preconditions.checkNotNull(key);
+      Preconditions.checkNotNull(iv);
+      contextReset = false;
+      cipher.init(mode, key, iv);
+    }
+    
+    /**
+     * 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)
+        throws IOException {
+      process(inBuffer, outBuffer);
+    }
+    
+    /**
+     * 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)
+        throws IOException {
+      process(inBuffer, outBuffer);
+    }
+    
+    private void process(ByteBuffer inBuffer, ByteBuffer outBuffer)
+        throws IOException {
+      try {
+        int inputSize = inBuffer.remaining();
+        // OpensslCipher#update will maintain crypto context.
+        int n = cipher.update(inBuffer, outBuffer);
+        if (n < inputSize) {
+          /**
+           * Typically code will not get here. OpensslCipher#update will 
+           * consume all input data and put result in outBuffer. 
+           * OpensslCipher#doFinal will reset the crypto context.
+           */
+          contextReset = true;
+          cipher.doFinal(outBuffer);
+        }
+      } catch (Exception e) {
+        throw new IOException(e);
+      }
+    }
+    
+    @Override
+    public boolean isContextReset() {
+      return contextReset;
+    }
+  }
+}

+ 213 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java

@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import java.nio.ByteBuffer;
+import java.security.NoSuchAlgorithmException;
+
+import javax.crypto.BadPaddingException;
+import javax.crypto.IllegalBlockSizeException;
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.ShortBufferException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.util.NativeCodeLoader;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * OpenSSL cipher using JNI.
+ * Currently only AES-CTR is supported. It's flexible to add 
+ * other crypto algorithms/modes.
+ */
+@InterfaceAudience.Private
+public final class OpensslCipher {
+  private static final Log LOG =
+      LogFactory.getLog(OpensslCipher.class.getName());
+  public static final int ENCRYPT_MODE = 1;
+  public static final int DECRYPT_MODE = 0;
+  
+  /** Currently only support AES/CTR/NoPadding. */
+  public static final int AES_CTR = 0;
+  public static final int PADDING_NOPADDING = 0;
+  
+  private long context = 0;
+  private final int algorithm;
+  private final int padding;
+  
+  private static boolean nativeCipherLoaded = false;
+  static {
+    if (NativeCodeLoader.isNativeCodeLoaded() &&
+        NativeCodeLoader.buildSupportsOpenssl()) {
+      try {
+        initIDs();
+        nativeCipherLoaded = true;
+      } catch (Throwable t) {
+        LOG.error("Failed to load OpenSSL Cipher.", t);
+      }
+    }
+  }
+  
+  public static boolean isNativeCodeLoaded() {
+    return nativeCipherLoaded;
+  }
+  
+  private OpensslCipher(long context, int algorithm, int padding) {
+    this.context = context;
+    this.algorithm = algorithm;
+    this.padding = padding;
+  }
+  
+  /**
+   * Return an <code>OpensslCipher<code> object that implements the specified
+   * algorithm.
+   * 
+   * @param algorithm currently only supports {@link #AES_CTR}
+   * @param padding currently only supports {@link #PADDING_NOPADDING}
+   * @return OpensslCipher an <code>OpensslCipher<code> object 
+   * @throws NoSuchAlgorithmException
+   * @throws NoSuchPaddingException
+   */
+  public static final OpensslCipher getInstance(int algorithm, 
+      int padding) throws NoSuchAlgorithmException, NoSuchPaddingException {
+    long context = initContext(algorithm, padding);
+    return new OpensslCipher(context, algorithm, padding);
+  }
+  
+  /**
+   * Initialize this cipher with a key and IV.
+   * 
+   * @param mode {@link #ENCRYPT_MODE} or {@link #DECRYPT_MODE}
+   * @param key crypto key
+   * @param iv crypto iv
+   */
+  public void init(int mode, byte[] key, byte[] iv) {
+    context = init(context, mode, algorithm, padding, key, iv);
+  }
+  
+  /**
+   * Continues a multiple-part encryption or decryption operation. The data
+   * is encrypted or decrypted, depending on how this cipher was initialized.
+   * <p/>
+   * 
+   * All <code>input.remaining()</code> bytes starting at 
+   * <code>input.position()</code> are processed. The result is stored in
+   * the output buffer.
+   * <p/>
+   * 
+   * Upon return, the input buffer's position will be equal to its limit;
+   * its limit will not have changed. The output buffer's position will have
+   * advanced by n, when n is the value returned by this method; the output
+   * buffer's limit will not have changed.
+   * <p/>
+   * 
+   * If <code>output.remaining()</code> bytes are insufficient to hold the
+   * result, a <code>ShortBufferException</code> is thrown.
+   * 
+   * @param input the input ByteBuffer
+   * @param output the output ByteBuffer
+   * @return int number of bytes stored in <code>output</code>
+   * @throws ShortBufferException if there is insufficient space in the
+   * output buffer
+   */
+  public int update(ByteBuffer input, ByteBuffer output) 
+      throws ShortBufferException {
+    checkState();
+    Preconditions.checkArgument(input.isDirect() && output.isDirect(), 
+        "Direct buffers are required.");
+    int len = update(context, input, input.position(), input.remaining(),
+        output, output.position(), output.remaining());
+    input.position(input.limit());
+    output.position(output.position() + len);
+    return len;
+  }
+  
+  /**
+   * Finishes a multiple-part operation. The data is encrypted or decrypted,
+   * depending on how this cipher was initialized.
+   * <p/>
+   * 
+   * The result is stored in the output buffer. Upon return, the output buffer's
+   * position will have advanced by n, where n is the value returned by this
+   * method; the output buffer's limit will not have changed.
+   * <p/>
+   * 
+   * If <code>output.remaining()</code> bytes are insufficient to hold the result,
+   * a <code>ShortBufferException</code> is thrown.
+   * <p/>
+   * 
+   * Upon finishing, this method resets this cipher object to the state it was
+   * in when previously initialized. That is, the object is available to encrypt
+   * or decrypt more data.
+   * <p/>
+   * 
+   * If any exception is thrown, this cipher object need to be reset before it 
+   * can be used again.
+   * 
+   * @param output the output ByteBuffer
+   * @return int number of bytes stored in <code>output</code>
+   * @throws ShortBufferException
+   * @throws IllegalBlockSizeException
+   * @throws BadPaddingException
+   */
+  public int doFinal(ByteBuffer output) throws ShortBufferException, 
+      IllegalBlockSizeException, BadPaddingException {
+    checkState();
+    Preconditions.checkArgument(output.isDirect(), "Direct buffer is required.");
+    int len = doFinal(context, output, output.position(), output.remaining());
+    output.position(output.position() + len);
+    return len;
+  }
+  
+  /** Forcibly clean the context. */
+  public void clean() {
+    if (context != 0) {
+      clean(context);
+      context = 0;
+    }
+  }
+
+  /** Check whether context is initialized. */
+  private void checkState() {
+    Preconditions.checkState(context != 0);
+  }
+  
+  @Override
+  protected void finalize() throws Throwable {
+    clean();
+  }
+
+  private native static void initIDs();
+  
+  private native static long initContext(int alg, int padding);
+  
+  private native long init(long context, int mode, int alg, int padding, 
+      byte[] key, byte[] iv);
+  
+  private native int update(long context, ByteBuffer input, int inputOffset, 
+      int inputLength, ByteBuffer output, int outputOffset, int maxOutputLength);
+  
+  private native int doFinal(long context, ByteBuffer output, int offset, 
+      int maxOutputLength);
+  
+  private native void clean(long context);
+  
+  public native static String getLibraryName();
+}

+ 382 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/OpensslCipher.c

@@ -0,0 +1,382 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+#include "org_apache_hadoop_crypto.h"
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+ 
+#include "org_apache_hadoop_crypto_OpensslCipher.h"
+
+#ifdef UNIX
+static EVP_CIPHER_CTX * (*dlsym_EVP_CIPHER_CTX_new)(void);
+static void (*dlsym_EVP_CIPHER_CTX_free)(EVP_CIPHER_CTX *);
+static int (*dlsym_EVP_CIPHER_CTX_cleanup)(EVP_CIPHER_CTX *);
+static void (*dlsym_EVP_CIPHER_CTX_init)(EVP_CIPHER_CTX *);
+static int (*dlsym_EVP_CIPHER_CTX_set_padding)(EVP_CIPHER_CTX *, int);
+static int (*dlsym_EVP_CipherInit_ex)(EVP_CIPHER_CTX *, const EVP_CIPHER *,  \
+           ENGINE *, const unsigned char *, const unsigned char *, int);
+static int (*dlsym_EVP_CipherUpdate)(EVP_CIPHER_CTX *, unsigned char *,  \
+           int *, const unsigned char *, int);
+static int (*dlsym_EVP_CipherFinal_ex)(EVP_CIPHER_CTX *, unsigned char *, int *);
+static EVP_CIPHER * (*dlsym_EVP_aes_256_ctr)(void);
+static EVP_CIPHER * (*dlsym_EVP_aes_128_ctr)(void);
+static void *openssl;
+#endif
+
+#ifdef WINDOWS
+typedef EVP_CIPHER_CTX * (__cdecl *__dlsym_EVP_CIPHER_CTX_new)(void);
+typedef void (__cdecl *__dlsym_EVP_CIPHER_CTX_free)(EVP_CIPHER_CTX *);
+typedef int (__cdecl *__dlsym_EVP_CIPHER_CTX_cleanup)(EVP_CIPHER_CTX *);
+typedef void (__cdecl *__dlsym_EVP_CIPHER_CTX_init)(EVP_CIPHER_CTX *);
+typedef int (__cdecl *__dlsym_EVP_CIPHER_CTX_set_padding)(EVP_CIPHER_CTX *, int);
+typedef int (__cdecl *__dlsym_EVP_CipherInit_ex)(EVP_CIPHER_CTX *,  \
+             const EVP_CIPHER *, ENGINE *, const unsigned char *,  \
+             const unsigned char *, int);
+typedef int (__cdecl *__dlsym_EVP_CipherUpdate)(EVP_CIPHER_CTX *,  \
+             unsigned char *, int *, const unsigned char *, int);
+typedef int (__cdecl *__dlsym_EVP_CipherFinal_ex)(EVP_CIPHER_CTX *,  \
+             unsigned char *, int *);
+typedef EVP_CIPHER * (__cdecl *__dlsym_EVP_aes_256_ctr)(void);
+typedef EVP_CIPHER * (__cdecl *__dlsym_EVP_aes_128_ctr)(void);
+static __dlsym_EVP_CIPHER_CTX_new dlsym_EVP_CIPHER_CTX_new;
+static __dlsym_EVP_CIPHER_CTX_free dlsym_EVP_CIPHER_CTX_free;
+static __dlsym_EVP_CIPHER_CTX_cleanup dlsym_EVP_CIPHER_CTX_cleanup;
+static __dlsym_EVP_CIPHER_CTX_init dlsym_EVP_CIPHER_CTX_init;
+static __dlsym_EVP_CIPHER_CTX_set_padding dlsym_EVP_CIPHER_CTX_set_padding;
+static __dlsym_EVP_CipherInit_ex dlsym_EVP_CipherInit_ex;
+static __dlsym_EVP_CipherUpdate dlsym_EVP_CipherUpdate;
+static __dlsym_EVP_CipherFinal_ex dlsym_EVP_CipherFinal_ex;
+static __dlsym_EVP_aes_256_ctr dlsym_EVP_aes_256_ctr;
+static __dlsym_EVP_aes_128_ctr dlsym_EVP_aes_128_ctr;
+static HMODULE openssl;
+#endif
+
+static void loadAesCtr(JNIEnv *env)
+{
+#ifdef UNIX
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_aes_256_ctr, env, openssl, "EVP_aes_256_ctr");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_aes_128_ctr, env, openssl, "EVP_aes_128_ctr");
+#endif
+
+#ifdef WINDOWS
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_aes_256_ctr, dlsym_EVP_aes_256_ctr,  \
+                      env, openssl, "EVP_aes_256_ctr");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_aes_128_ctr, dlsym_EVP_aes_128_ctr,  \
+                      env, openssl, "EVP_aes_128_ctr");
+#endif
+}
+
+JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initIDs
+    (JNIEnv *env, jclass clazz)
+{
+  char msg[1000];
+#ifdef UNIX
+  openssl = dlopen(HADOOP_OPENSSL_LIBRARY, RTLD_LAZY | RTLD_GLOBAL);
+#endif
+
+#ifdef WINDOWS
+  openssl = LoadLibrary(HADOOP_OPENSSL_LIBRARY);
+#endif
+
+  if (!openssl) {
+    snprintf(msg, sizeof(msg), "Cannot load %s (%s)!", HADOOP_OPENSSL_LIBRARY,  \
+        dlerror());
+    THROW(env, "java/lang/UnsatisfiedLinkError", msg);
+    return;
+  }
+
+#ifdef UNIX
+  dlerror();  // Clear any existing error
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_new, env, openssl,  \
+                      "EVP_CIPHER_CTX_new");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_free, env, openssl,  \
+                      "EVP_CIPHER_CTX_free");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_cleanup, env, openssl,  \
+                      "EVP_CIPHER_CTX_cleanup");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_init, env, openssl,  \
+                      "EVP_CIPHER_CTX_init");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CIPHER_CTX_set_padding, env, openssl,  \
+                      "EVP_CIPHER_CTX_set_padding");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CipherInit_ex, env, openssl,  \
+                      "EVP_CipherInit_ex");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CipherUpdate, env, openssl,  \
+                      "EVP_CipherUpdate");
+  LOAD_DYNAMIC_SYMBOL(dlsym_EVP_CipherFinal_ex, env, openssl,  \
+                      "EVP_CipherFinal_ex");
+#endif
+
+#ifdef WINDOWS
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_new, dlsym_EVP_CIPHER_CTX_new,  \
+                      env, openssl, "EVP_CIPHER_CTX_new");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_free, dlsym_EVP_CIPHER_CTX_free,  \
+                      env, openssl, "EVP_CIPHER_CTX_free");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_cleanup,  \
+                      dlsym_EVP_CIPHER_CTX_cleanup, env, 
+                      openssl, "EVP_CIPHER_CTX_cleanup");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_init, dlsym_EVP_CIPHER_CTX_init,  \
+                      env, openssl, "EVP_CIPHER_CTX_init");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CIPHER_CTX_set_padding,  \
+                      dlsym_EVP_CIPHER_CTX_set_padding, env,  \
+                      openssl, "EVP_CIPHER_CTX_set_padding");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CipherInit_ex, dlsym_EVP_CipherInit_ex,  \
+                      env, openssl, "EVP_CipherInit_ex");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CipherUpdate, dlsym_EVP_CipherUpdate,  \
+                      env, openssl, "EVP_CipherUpdate");
+  LOAD_DYNAMIC_SYMBOL(__dlsym_EVP_CipherFinal_ex, dlsym_EVP_CipherFinal_ex,  \
+                      env, openssl, "EVP_CipherFinal_ex");
+#endif
+
+  loadAesCtr(env);
+  jthrowable jthr = (*env)->ExceptionOccurred(env);
+  if (jthr) {
+    (*env)->DeleteLocalRef(env, jthr);
+    THROW(env, "java/lang/UnsatisfiedLinkError",  \
+        "Cannot find AES-CTR support, is your version of Openssl new enough?");
+    return;
+  }
+}
+
+JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_initContext
+    (JNIEnv *env, jclass clazz, jint alg, jint padding)
+{
+  if (alg != AES_CTR) {
+    THROW(env, "java/security/NoSuchAlgorithmException", NULL);
+    return (jlong)0;
+  }
+  if (padding != NOPADDING) {
+    THROW(env, "javax/crypto/NoSuchPaddingException", NULL);
+    return (jlong)0;
+  }
+  
+  if (dlsym_EVP_aes_256_ctr == NULL || dlsym_EVP_aes_128_ctr == NULL) {
+    THROW(env, "java/security/NoSuchAlgorithmException",  \
+        "Doesn't support AES CTR.");
+    return (jlong)0;
+  }
+  
+  // Create and initialize a EVP_CIPHER_CTX
+  EVP_CIPHER_CTX *context = dlsym_EVP_CIPHER_CTX_new();
+  if (!context) {
+    THROW(env, "java/lang/OutOfMemoryError", NULL);
+    return (jlong)0;
+  }
+   
+  return JLONG(context);
+}
+
+// Only supports AES-CTR currently
+static EVP_CIPHER * getEvpCipher(int alg, int keyLen)
+{
+  EVP_CIPHER *cipher = NULL;
+  if (alg == AES_CTR) {
+    if (keyLen == KEY_LENGTH_256) {
+      cipher = dlsym_EVP_aes_256_ctr();
+    } else if (keyLen == KEY_LENGTH_128) {
+      cipher = dlsym_EVP_aes_128_ctr();
+    }
+  }
+  return cipher;
+}
+
+JNIEXPORT jlong JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_init
+    (JNIEnv *env, jobject object, jlong ctx, jint mode, jint alg, jint padding, 
+    jbyteArray key, jbyteArray iv)
+{
+  int jKeyLen = (*env)->GetArrayLength(env, key);
+  int jIvLen = (*env)->GetArrayLength(env, iv);
+  if (jKeyLen != KEY_LENGTH_128 && jKeyLen != KEY_LENGTH_256) {
+    THROW(env, "java/lang/IllegalArgumentException", "Invalid key length.");
+    return (jlong)0;
+  }
+  if (jIvLen != IV_LENGTH) {
+    THROW(env, "java/lang/IllegalArgumentException", "Invalid iv length.");
+    return (jlong)0;
+  }
+  
+  EVP_CIPHER_CTX *context = CONTEXT(ctx);
+  if (context == 0) {
+    // Create and initialize a EVP_CIPHER_CTX
+    context = dlsym_EVP_CIPHER_CTX_new();
+    if (!context) {
+      THROW(env, "java/lang/OutOfMemoryError", NULL);
+      return (jlong)0;
+    }
+  }
+  
+  jbyte *jKey = (*env)->GetByteArrayElements(env, key, NULL);
+  if (jKey == NULL) {
+    THROW(env, "java/lang/InternalError", "Cannot get bytes array for key.");
+    return (jlong)0;
+  }
+  jbyte *jIv = (*env)->GetByteArrayElements(env, iv, NULL);
+  if (jIv == NULL) {
+    (*env)->ReleaseByteArrayElements(env, key, jKey, 0);
+    THROW(env, "java/lang/InternalError", "Cannot get bytes array for iv.");
+    return (jlong)0;
+  }
+  
+  int rc = dlsym_EVP_CipherInit_ex(context, getEvpCipher(alg, jKeyLen),  \
+      NULL, (unsigned char *)jKey, (unsigned char *)jIv, mode == ENCRYPT_MODE);
+  (*env)->ReleaseByteArrayElements(env, key, jKey, 0);
+  (*env)->ReleaseByteArrayElements(env, iv, jIv, 0);
+  if (rc == 0) {
+    dlsym_EVP_CIPHER_CTX_cleanup(context);
+    THROW(env, "java/lang/InternalError", "Error in EVP_CipherInit_ex.");
+    return (jlong)0;
+  }
+  
+  if (padding == NOPADDING) {
+    dlsym_EVP_CIPHER_CTX_set_padding(context, 0);
+  }
+  
+  return JLONG(context);
+}
+
+// https://www.openssl.org/docs/crypto/EVP_EncryptInit.html
+static int check_update_max_output_len(EVP_CIPHER_CTX *context, int input_len, 
+    int max_output_len)
+{
+  if (context->flags & EVP_CIPH_NO_PADDING) {
+    if (max_output_len >= input_len) {
+      return 1;
+    }
+    return 0;
+  } else {
+    int b = context->cipher->block_size;
+    if (context->encrypt) {
+      if (max_output_len >= input_len + b - 1) {
+        return 1;
+      }
+    } else {
+      if (max_output_len >= input_len + b) {
+        return 1;
+      }
+    }
+    
+    return 0;
+  }
+}
+
+JNIEXPORT jint JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_update
+    (JNIEnv *env, jobject object, jlong ctx, jobject input, jint input_offset, 
+    jint input_len, jobject output, jint output_offset, jint max_output_len)
+{
+  EVP_CIPHER_CTX *context = CONTEXT(ctx);
+  if (!check_update_max_output_len(context, input_len, max_output_len)) {
+    THROW(env, "javax/crypto/ShortBufferException",  \
+        "Output buffer is not sufficient.");
+    return 0;
+  }
+  unsigned char *input_bytes = (*env)->GetDirectBufferAddress(env, input);
+  unsigned char *output_bytes = (*env)->GetDirectBufferAddress(env, output);
+  if (input_bytes == NULL || output_bytes == NULL) {
+    THROW(env, "java/lang/InternalError", "Cannot get buffer address.");
+    return 0;
+  }
+  input_bytes = input_bytes + input_offset;
+  output_bytes = output_bytes + output_offset;
+  
+  int output_len = 0;
+  if (!dlsym_EVP_CipherUpdate(context, output_bytes, &output_len,  \
+      input_bytes, input_len)) {
+    dlsym_EVP_CIPHER_CTX_cleanup(context);
+    THROW(env, "java/lang/InternalError", "Error in EVP_CipherUpdate.");
+    return 0;
+  }
+  return output_len;
+}
+
+// https://www.openssl.org/docs/crypto/EVP_EncryptInit.html
+static int check_doFinal_max_output_len(EVP_CIPHER_CTX *context, 
+    int max_output_len)
+{
+  if (context->flags & EVP_CIPH_NO_PADDING) {
+    return 1;
+  } else {
+    int b = context->cipher->block_size;
+    if (max_output_len >= b) {
+      return 1;
+    }
+    
+    return 0;
+  }
+}
+
+JNIEXPORT jint JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_doFinal
+    (JNIEnv *env, jobject object, jlong ctx, jobject output, jint offset, 
+    jint max_output_len)
+{
+  EVP_CIPHER_CTX *context = CONTEXT(ctx);
+  if (!check_doFinal_max_output_len(context, max_output_len)) {
+    THROW(env, "javax/crypto/ShortBufferException",  \
+        "Output buffer is not sufficient.");
+    return 0;
+  }
+  unsigned char *output_bytes = (*env)->GetDirectBufferAddress(env, output);
+  if (output_bytes == NULL) {
+    THROW(env, "java/lang/InternalError", "Cannot get buffer address.");
+    return 0;
+  }
+  output_bytes = output_bytes + offset;
+  
+  int output_len = 0;
+  if (!dlsym_EVP_CipherFinal_ex(context, output_bytes, &output_len)) {
+    dlsym_EVP_CIPHER_CTX_cleanup(context);
+    THROW(env, "java/lang/InternalError", "Error in EVP_CipherFinal_ex.");
+    return 0;
+  }
+  return output_len;
+}
+
+JNIEXPORT void JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_clean
+    (JNIEnv *env, jobject object, jlong ctx) 
+{
+  EVP_CIPHER_CTX *context = CONTEXT(ctx);
+  if (context) {
+    dlsym_EVP_CIPHER_CTX_free(context);
+  }
+}
+
+JNIEXPORT jstring JNICALL Java_org_apache_hadoop_crypto_OpensslCipher_getLibraryName
+    (JNIEnv *env, jclass clazz) 
+{
+#ifdef UNIX
+  if (dlsym_EVP_CIPHER_CTX_init) {
+    Dl_info dl_info;
+    if(dladdr(
+        dlsym_EVP_CIPHER_CTX_init,
+        &dl_info)) {
+      return (*env)->NewStringUTF(env, dl_info.dli_fname);
+    }
+  }
+
+  return (*env)->NewStringUTF(env, HADOOP_OPENSSL_LIBRARY);
+#endif
+
+#ifdef WINDOWS
+  LPWSTR filename = NULL;
+  GetLibraryName(dlsym_EVP_CIPHER_CTX_init, &filename);
+  if (filename != NULL) {
+    return (*env)->NewString(env, filename, (jsize) wcslen(filename));
+  } else {
+    return (*env)->NewStringUTF(env, "Unavailable");
+  }
+#endif
+}

+ 61 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/crypto/org_apache_hadoop_crypto.h

@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+ 
+#ifndef ORG_APACHE_HADOOP_CRYPTO_H
+#define ORG_APACHE_HADOOP_CRYPTO_H
+
+#include "org_apache_hadoop.h"
+
+#ifdef UNIX
+#include <dlfcn.h>
+#include "config.h"
+#endif
+
+#ifdef WINDOWS
+#include "winutils.h"
+#endif
+
+#include <openssl/aes.h>
+#include <openssl/evp.h>
+#include <openssl/err.h>
+
+/**
+ * A helper macro to convert the java 'context-handle' 
+ * to a EVP_CIPHER_CTX pointer. 
+ */
+#define CONTEXT(context) ((EVP_CIPHER_CTX*)((ptrdiff_t)(context)))
+
+/**
+ * A helper macro to convert the EVP_CIPHER_CTX pointer to the 
+ * java 'context-handle'.
+ */
+#define JLONG(context) ((jlong)((ptrdiff_t)(context)))
+
+#define KEY_LENGTH_128 16
+#define KEY_LENGTH_256 32
+#define IV_LENGTH 16
+
+#define ENCRYPT_MODE 1
+#define DECRYPT_MODE 0
+
+/** Currently only support AES/CTR/NoPadding. */
+#define AES_CTR 0
+#define NOPADDING 0
+#define PKCSPADDING 1
+
+#endif //ORG_APACHE_HADOOP_CRYPTO_H

+ 35 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsWithOpensslAesCtrCryptoCodec.java

@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.BeforeClass;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASS_KEY;
+
+public class TestCryptoStreamsWithOpensslAesCtrCryptoCodec 
+    extends TestCryptoStreams {
+  
+  @BeforeClass
+  public static void init() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(HADOOP_SECURITY_CRYPTO_CODEC_CLASS_KEY, 
+        OpensslAesCtrCryptoCodec.class.getName());
+    codec = CryptoCodec.getInstance(conf);
+  }
+}

+ 120 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestOpensslCipher.java

@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.crypto;
+
+import java.nio.ByteBuffer;
+import java.security.NoSuchAlgorithmException;
+
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.ShortBufferException;
+
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestOpensslCipher {
+  private 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, 
+    0x07, 0x08, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08};
+  
+  @Test(timeout=120000)
+  public void testGetInstance() throws Exception {
+    if (!OpensslCipher.isNativeCodeLoaded()) {
+      return;
+    }
+    OpensslCipher cipher = OpensslCipher.getInstance(OpensslCipher.AES_CTR, 
+        OpensslCipher.PADDING_NOPADDING);
+    Assert.assertTrue(cipher != null);
+    
+    try {
+      cipher = OpensslCipher.getInstance(OpensslCipher.AES_CTR + 100, 
+          OpensslCipher.PADDING_NOPADDING);
+      Assert.fail("Should specify correct algorithm.");
+    } catch (NoSuchAlgorithmException e) {
+      // Expect NoSuchAlgorithmException
+    }
+    
+    try {
+      cipher = OpensslCipher.getInstance(OpensslCipher.AES_CTR, 
+          OpensslCipher.PADDING_NOPADDING + 100);
+      Assert.fail("Should specify correct padding.");
+    } catch (NoSuchPaddingException e) {
+      // Expect NoSuchPaddingException
+    }
+  }
+  
+  @Test(timeout=120000)
+  public void testUpdateArguments() throws Exception {
+    if (!OpensslCipher.isNativeCodeLoaded()) {
+      return;
+    }
+    OpensslCipher cipher = OpensslCipher.getInstance(OpensslCipher.AES_CTR, 
+        OpensslCipher.PADDING_NOPADDING);
+    Assert.assertTrue(cipher != null);
+    
+    cipher.init(OpensslCipher.ENCRYPT_MODE, key, iv);
+    
+    // Require direct buffers
+    ByteBuffer input = ByteBuffer.allocate(1024);
+    ByteBuffer output = ByteBuffer.allocate(1024);
+    
+    try {
+      cipher.update(input, output);
+      Assert.fail("Input and output buffer should be direct buffer.");
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Direct buffers are required", e);
+    }
+    
+    // Output buffer length should be sufficient to store output data 
+    input = ByteBuffer.allocateDirect(1024);
+    output = ByteBuffer.allocateDirect(1000);
+    try {
+      cipher.update(input, output);
+      Assert.fail("Output buffer length should be sufficient " +
+          "to store output data");
+    } catch (ShortBufferException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Output buffer is not sufficient", e);
+    }
+  }
+  
+  @Test(timeout=120000)
+  public void testDoFinalArguments() throws Exception {
+    if (!OpensslCipher.isNativeCodeLoaded()) {
+      return;
+    }
+    OpensslCipher cipher = OpensslCipher.getInstance(OpensslCipher.AES_CTR, 
+        OpensslCipher.PADDING_NOPADDING);
+    Assert.assertTrue(cipher != null);
+    
+    cipher.init(OpensslCipher.ENCRYPT_MODE, key, iv);
+    
+    // Require direct buffer
+    ByteBuffer output = ByteBuffer.allocate(1024);
+    
+    try {
+      cipher.doFinal(output);
+      Assert.fail("Output buffer should be direct buffer.");
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Direct buffer is required", e);
+    }
+  }
+}