You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/12/07 07:20:46 UTC

[2/3] hbase git commit: HBASE-12606 Sanity check encryption configuration before opening WAL or onlining regions

HBASE-12606 Sanity check encryption configuration before opening WAL or onlining regions

Also disables IntegrationTestIngestWithEncryption if distributed
cluster configuration is missing prerequisites.


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

Branch: refs/heads/branch-1
Commit: 7b7b39805620aa74839cf42f711f6e9fe92f873f
Parents: 6c4863f
Author: Andrew Purtell <ap...@apache.org>
Authored: Sat Dec 6 21:19:14 2014 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Sat Dec 6 22:04:26 2014 -0800

----------------------------------------------------------------------
 .../IntegrationTestIngestWithEncryption.java    |  36 ++++-
 .../org/apache/hadoop/hbase/master/HMaster.java |  38 ++++-
 .../hadoop/hbase/regionserver/HRegion.java      |   9 +-
 .../wal/SecureProtobufLogReader.java            |   5 +-
 .../wal/SecureProtobufLogWriter.java            |   4 +
 .../hadoop/hbase/util/EncryptionTest.java       | 158 +++++++++++++++++++
 .../hadoop/hbase/util/TestEncryptionTest.java   | 139 ++++++++++++++++
 7 files changed, 381 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/7b7b3980/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java
index f4f5a2c..220caf0 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestIngestWithEncryption.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.wal.WAL.Reader;
 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
 import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader;
 import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter;
+import org.apache.hadoop.hbase.util.EncryptionTest;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -39,6 +40,7 @@ import org.junit.experimental.categories.Category;
 
 @Category(IntegrationTests.class)
 public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
+  boolean initialized = false;
 
   static {
     // These log level changes are only useful when running on a localhost
@@ -53,11 +55,9 @@ public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
   public void setUpCluster() throws Exception {
     util = getTestingUtil(null);
     Configuration conf = util.getConfiguration();
-    conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
     if (!util.isDistributedCluster()) {
-      // Inject the test key provider and WAL alternative if running on a
-      // localhost cluster; otherwise, whether or not the schema change below
-      // takes effect depends on the distributed cluster site configuration.
+      // Inject required configuration if we are not running in distributed mode
+      conf.setInt(HFile.FORMAT_VERSION_KEY, 3);
       conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
       conf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
       conf.setClass("hbase.regionserver.hlog.reader.impl", SecureProtobufLogReader.class,
@@ -66,7 +66,15 @@ public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
         Writer.class);
       conf.setBoolean(HConstants.ENABLE_WAL_ENCRYPTION, true);
     }
+    // Check if the cluster configuration can support this test
+    try {
+      EncryptionTest.testEncryption(conf, "AES", null);
+    } catch (Exception e) {
+      LOG.warn("Encryption configuration test did not pass, skipping test");
+      return;
+    }
     super.setUpCluster();
+    initialized = true;
   }
 
   @Before
@@ -76,6 +84,10 @@ public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
     // will create the test table, appropriately pre-split
     super.setUp();
 
+    if (!initialized) {
+      return;
+    }
+
     // Update the test table schema so HFiles from this point will be written with
     // encryption features enabled.
     final Admin admin = util.getHBaseAdmin();
@@ -97,6 +109,22 @@ public class IntegrationTestIngestWithEncryption extends IntegrationTestIngest {
     }
   }
 
+  @Override
+  public int runTestFromCommandLine() throws Exception {
+    if (!initialized) {
+      return 0;
+    }
+    return super.runTestFromCommandLine();
+  }
+
+  @Override
+  public void cleanUp() throws Exception {
+    if (!initialized) {
+      return;
+    }
+    super.cleanUp();
+  }
+
   public static void main(String[] args) throws Exception {
     Configuration conf = HBaseConfiguration.create();
     IntegrationTestingUtility.setUseDistributedCluster(conf);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7b7b3980/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index bc4815d..c27ac1b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -113,6 +113,7 @@ import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.ConfigUtil;
+import org.apache.hadoop.hbase.util.EncryptionTest;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.hbase.util.Pair;
@@ -222,6 +223,9 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
   //should we check the compression codec type at master side, default true, HBASE-6370
   private final boolean masterCheckCompression;
 
+  //should we check encryption settings at master side, default true
+  private final boolean masterCheckEncryption;
+
   Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
 
   // monitor for snapshot of hbase tables
@@ -283,9 +287,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       this.conf.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName.toString());
     }
 
-    //should we check the compression codec type at master side, default true, HBASE-6370
+    // should we check the compression codec type at master side, default true, HBASE-6370
     this.masterCheckCompression = conf.getBoolean("hbase.master.check.compression", true);
 
+    // should we check encryption settings at master side, default true
+    this.masterCheckEncryption = conf.getBoolean("hbase.master.check.encryption", true);
+
     this.metricsMaster = new MetricsMaster( new MetricsMasterWrapperImpl(this));
 
     // preload table descriptor at startup
@@ -1245,7 +1252,18 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     }
 
     // check compression can be loaded
-    checkCompression(htd);
+    try {
+      checkCompression(htd);
+    } catch (IOException e) {
+      throw new DoNotRetryIOException(e.getMessage(), e);
+    }
+
+    // check encryption can be loaded
+    try {
+      checkEncryption(conf, htd);
+    } catch (IOException e) {
+      throw new DoNotRetryIOException(e.getMessage(), e);
+    }
 
     // check that we have at least 1 CF
     if (htd.getColumnFamilies().length == 0) {
@@ -1367,6 +1385,20 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
     CompressionTest.testCompression(hcd.getCompactionCompression());
   }
 
+  private void checkEncryption(final Configuration conf, final HTableDescriptor htd)
+  throws IOException {
+    if (!this.masterCheckEncryption) return;
+    for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
+      checkEncryption(conf, hcd);
+    }
+  }
+
+  private void checkEncryption(final Configuration conf, final HColumnDescriptor hcd)
+  throws IOException {
+    if (!this.masterCheckEncryption) return;
+    EncryptionTest.testEncryption(conf, hcd.getEncryptionType(), hcd.getEncryptionKey());
+  }
+
   private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
     byte[][] splitKeys) {
     long regionId = System.currentTimeMillis();
@@ -1427,6 +1459,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       throws IOException {
     checkInitialized();
     checkCompression(columnDescriptor);
+    checkEncryption(conf, columnDescriptor);
     if (cpHost != null) {
       if (cpHost.preAddColumn(tableName, columnDescriptor)) {
         return;
@@ -1444,6 +1477,7 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
       throws IOException {
     checkInitialized();
     checkCompression(descriptor);
+    checkEncryption(conf, descriptor);
     if (cpHost != null) {
       if (cpHost.preModifyColumn(tableName, descriptor)) {
         return;

http://git-wip-us.apache.org/repos/asf/hbase/blob/7b7b3980/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 67cd70b..5000663 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -147,6 +147,7 @@ import org.apache.hadoop.hbase.util.CancelableProgressable;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.hbase.util.CompressionTest;
 import org.apache.hadoop.hbase.util.Counter;
+import org.apache.hadoop.hbase.util.EncryptionTest;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.FSTableDescriptors;
 import org.apache.hadoop.hbase.util.FSUtils;
@@ -4855,7 +4856,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
   protected HRegion openHRegion(final CancelableProgressable reporter)
   throws IOException {
     checkCompressionCodecs();
-
+    checkEncryption();
     this.openSeqNum = initialize(reporter);
     this.setSequenceId(openSeqNum);
     if (wal != null && getRegionServerServices() != null) {
@@ -4871,6 +4872,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver { //
     }
   }
 
+  private void checkEncryption() throws IOException {
+    for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
+      EncryptionTest.testEncryption(conf, fam.getEncryptionType(), fam.getEncryptionKey());
+    }
+  }
+
   /**
    * Create a daughter region from given a temp directory with the region data.
    * @param hri Spec. for daughter region to open.

http://git-wip-us.apache.org/repos/asf/hbase/blob/7b7b3980/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogReader.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogReader.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogReader.java
index 2fafabb..041dfe2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogReader.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogReader.java
@@ -34,9 +34,9 @@ import org.apache.hadoop.hbase.io.crypto.Cipher;
 import org.apache.hadoop.hbase.io.crypto.Decryptor;
 import org.apache.hadoop.hbase.io.crypto.Encryption;
 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader;
-import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.WALHdrResult;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
 import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.util.EncryptionTest;
 
 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
 public class SecureProtobufLogReader extends ProtobufLogReader {
@@ -67,6 +67,9 @@ public class SecureProtobufLogReader extends ProtobufLogReader {
       // Serialized header data has been merged into the builder from the
       // stream.
 
+      EncryptionTest.testKeyProvider(conf);
+      EncryptionTest.testCipherProvider(conf);
+
       // Retrieve a usable key
 
       byte[] keyBytes = builder.getEncryptionKey().toByteArray();

http://git-wip-us.apache.org/repos/asf/hbase/blob/7b7b3980/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java
index 03d1608..e850485 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SecureProtobufLogWriter.java
@@ -26,6 +26,7 @@ import javax.crypto.spec.SecretKeySpec;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.util.EncryptionTest;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -51,6 +52,9 @@ public class SecureProtobufLogWriter extends ProtobufLogWriter {
       throws IOException {
     builder.setWriterClsName(SecureProtobufLogWriter.class.getSimpleName());
     if (conf.getBoolean(HConstants.ENABLE_WAL_ENCRYPTION, false)) {
+      EncryptionTest.testKeyProvider(conf);
+      EncryptionTest.testCipherProvider(conf);
+
       // Get an instance of our cipher
       final String cipherName = conf.get(HConstants.CRYPTO_WAL_ALGORITHM_CONF_KEY, DEFAULT_CIPHER);
       Cipher cipher = Encryption.getCipher(conf, cipherName);

http://git-wip-us.apache.org/repos/asf/hbase/blob/7b7b3980/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
new file mode 100644
index 0000000..1cd6ead
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/EncryptionTest.java
@@ -0,0 +1,158 @@
+/**
+ *
+ * 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.hbase.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.crypto.DefaultCipherProvider;
+import org.apache.hadoop.hbase.io.crypto.Encryption;
+import org.apache.hadoop.hbase.io.crypto.KeyStoreKeyProvider;
+import org.apache.hadoop.hbase.security.EncryptionUtil;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class EncryptionTest {
+  static final Log LOG = LogFactory.getLog(EncryptionTest.class);
+
+  static final Map<String, Boolean> keyProviderResults = new ConcurrentHashMap<String, Boolean>();
+  static final Map<String, Boolean> cipherProviderResults =
+    new ConcurrentHashMap<String, Boolean>();
+  static final Map<String, Boolean> cipherResults = new ConcurrentHashMap<String, Boolean>();
+
+  private EncryptionTest() {
+  }
+
+  /**
+   * Check that the configured key provider can be loaded and initialized, or
+   * throw an exception.
+   *
+   * @param conf
+   * @throws IOException
+   */
+  public static void testKeyProvider(final Configuration conf) throws IOException {
+    String providerClassName = conf.get(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY,
+      KeyStoreKeyProvider.class.getName());
+    Boolean result = keyProviderResults.get(providerClassName);
+    if (result == null) {
+      try {
+        Encryption.getKeyProvider(conf);
+        keyProviderResults.put(providerClassName, true);
+      } catch (Exception e) { // most likely a RuntimeException
+        keyProviderResults.put(providerClassName, false);
+        throw new IOException("Key provider " + providerClassName + " failed test: " +
+          e.getMessage(), e);
+      }
+    } else if (result.booleanValue() == false) {
+      throw new IOException("Key provider " + providerClassName + " previously failed test");
+    }
+  }
+
+  /**
+   * Check that the configured cipher provider can be loaded and initialized, or
+   * throw an exception.
+   *
+   * @param conf
+   * @throws IOException
+   */
+  public static void testCipherProvider(final Configuration conf) throws IOException {
+    String providerClassName = conf.get(HConstants.CRYPTO_CIPHERPROVIDER_CONF_KEY,
+      DefaultCipherProvider.class.getName());
+    Boolean result = cipherProviderResults.get(providerClassName);
+    if (result == null) {
+      try {
+        Encryption.getCipherProvider(conf);
+        cipherProviderResults.put(providerClassName, true);
+      } catch (Exception e) { // most likely a RuntimeException
+        cipherProviderResults.put(providerClassName, false);
+        throw new IOException("Cipher provider " + providerClassName + " failed test: " +
+          e.getMessage(), e);
+      }
+    } else if (result.booleanValue() == false) {
+      throw new IOException("Cipher provider " + providerClassName + " previously failed test");
+    }
+  }
+
+  /**
+   * Check that the specified cipher can be loaded and initialized, or throw
+   * an exception. Verifies key and cipher provider configuration as a
+   * prerequisite for cipher verification.
+   *
+   * @param conf
+   * @param cipher
+   * @param key
+   * @throws IOException
+   */
+  public static void testEncryption(final Configuration conf, final String cipher,
+      byte[] key) throws IOException {
+    if (cipher == null) {
+      return;
+    }
+    testKeyProvider(conf);
+    testCipherProvider(conf);
+    Boolean result = cipherResults.get(cipher);
+    if (result == null) {
+      try {
+        Encryption.Context context = Encryption.newContext(conf);
+        context.setCipher(Encryption.getCipher(conf, cipher));
+        if (key == null) {
+          // Make a random key since one was not provided
+          context.setKey(context.getCipher().getRandomKey());
+        } else {
+          // This will be a wrapped key from schema
+          context.setKey(EncryptionUtil.unwrapKey(conf,
+            conf.get(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase"),
+            key));
+        }
+        byte[] iv = null;
+        if (context.getCipher().getIvLength() > 0) {
+          iv = new byte[context.getCipher().getIvLength()];
+          Bytes.random(iv);
+        }
+        byte[] plaintext = new byte[1024];
+        Bytes.random(plaintext);
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        Encryption.encrypt(out, new ByteArrayInputStream(plaintext), context, iv);
+        byte[] ciphertext = out.toByteArray();
+        out.reset();
+        Encryption.decrypt(out, new ByteArrayInputStream(ciphertext), plaintext.length,
+          context, iv);
+        byte[] test = out.toByteArray();
+        if (!Bytes.equals(plaintext, test)) {
+          throw new IOException("Did not pass encrypt/decrypt test");
+        }
+        cipherResults.put(cipher, true);
+      } catch (Exception e) {
+        cipherResults.put(cipher, false);
+        throw new IOException("Cipher " + cipher + " failed test: " + e.getMessage(), e);
+      }
+    } else if (result.booleanValue() == false) {
+      throw new IOException("Cipher " + cipher + " previously failed test");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/7b7b3980/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java
new file mode 100644
index 0000000..544d775
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestEncryptionTest.java
@@ -0,0 +1,139 @@
+/*
+ *
+ * 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.hbase.util;
+
+import static org.junit.Assert.fail;
+
+import java.security.Key;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.io.crypto.Cipher;
+import org.apache.hadoop.hbase.io.crypto.CipherProvider;
+import org.apache.hadoop.hbase.io.crypto.DefaultCipherProvider;
+import org.apache.hadoop.hbase.io.crypto.KeyProvider;
+import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestEncryptionTest {
+
+  @Test
+  public void testTestKeyProvider() {
+    Configuration conf = HBaseConfiguration.create();
+    try {
+      conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
+      EncryptionTest.testKeyProvider(conf);
+    } catch (Exception e) {
+      fail("Instantiation of test key provider should have passed");
+    }
+    try {
+      conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, FailingKeyProvider.class.getName());
+      EncryptionTest.testKeyProvider(conf);
+      fail("Instantiation of bad test key provider should have failed check");
+    } catch (Exception e) { }
+  }
+
+  @Test
+  public void testTestCipherProvider() {
+    Configuration conf = HBaseConfiguration.create();
+    try {
+      conf.set(HConstants.CRYPTO_CIPHERPROVIDER_CONF_KEY, DefaultCipherProvider.class.getName());
+      EncryptionTest.testCipherProvider(conf);
+    } catch (Exception e) {
+      fail("Instantiation of test cipher provider should have passed");
+    }
+    try {
+      conf.set(HConstants.CRYPTO_CIPHERPROVIDER_CONF_KEY, FailingCipherProvider.class.getName());
+      EncryptionTest.testCipherProvider(conf);
+      fail("Instantiation of bad test cipher provider should have failed check");
+    } catch (Exception e) { }
+  }
+
+  @Test
+  public void testTestCipher() {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
+    try {
+      EncryptionTest.testEncryption(conf, "AES", null);
+    } catch (Exception e) {
+      fail("Test for cipher AES should have succeeded");
+    }
+    try {
+      EncryptionTest.testEncryption(conf, "foobar", null);
+      fail("Test for bogus cipher should have failed");
+    } catch (Exception e) { }
+  }
+
+  public static class FailingKeyProvider implements KeyProvider {
+
+    @Override
+    public void init(String params) {
+      throw new RuntimeException("BAD!");
+    }
+
+    @Override
+    public Key getKey(String alias) {
+      return null;
+    }
+
+    @Override
+    public Key[] getKeys(String[] aliases) {
+      return null;
+    }
+
+  }
+
+  public static class FailingCipherProvider implements CipherProvider {
+
+    public FailingCipherProvider() {
+      super();
+      throw new RuntimeException("BAD!");
+    }
+
+    @Override
+    public Configuration getConf() {
+      return null;
+    }
+
+    @Override
+    public void setConf(Configuration conf) {
+    }
+
+    @Override
+    public String getName() {
+      return null;
+    }
+
+    @Override
+    public String[] getSupportedCiphers() {
+      return null;
+    }
+
+    @Override
+    public Cipher getCipher(String name) {
+      return null;
+    }
+    
+  }
+}