You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/08/06 14:37:44 UTC

[8/9] cassandra git commit: Add transparent data encryption core classes (CASSANDRA-9945)

Add transparent data encryption core classes (CASSANDRA-9945)


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

Branch: refs/heads/cassandra-3.0
Commit: 765ab3fc3b9f4e891c257f36f117ddf4232da6be
Parents: c3ed25b
Author: Jason Brown <ja...@gmail.com>
Authored: Thu Jul 30 14:03:17 2015 -0700
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Thu Aug 6 14:36:07 2015 +0200

----------------------------------------------------------------------
 .../org/apache/cassandra/config/Config.java     |   3 +-
 .../cassandra/config/DatabaseDescriptor.java    |  17 ++
 .../TransparentDataEncryptionOptions.java       |  76 ++++++++
 .../cassandra/security/CipherFactory.java       | 175 +++++++++++++++++++
 .../cassandra/security/EncryptionContext.java   | 122 +++++++++++++
 .../cassandra/security/JKSKeyProvider.java      |  90 ++++++++++
 .../apache/cassandra/security/KeyProvider.java  |  33 ++++
 test/conf/cassandra.keystore                    | Bin 0 -> 1004 bytes
 test/conf/cassandra_encryption.yaml             |  14 ++
 .../cassandra/security/CipherFactoryTest.java   |  87 +++++++++
 .../security/EncryptionContextGenerator.java    |  54 ++++++
 .../cassandra/security/JKSKeyProviderTest.java  |  52 ++++++
 12 files changed, 722 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index e93d090..f8f34e0 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -169,7 +169,8 @@ public class Config
     public int commitlog_segment_size_in_mb = 32;
     public ParameterizedClass commitlog_compression;
     public int commitlog_max_compression_buffers_in_pool = 3;
- 
+    public TransparentDataEncryptionOptions transparent_data_encryption_options = new TransparentDataEncryptionOptions();
+
     @Deprecated
     public int commitlog_periodic_queue_size = -1;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 02f3c17..e7b9455 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.locator.*;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.scheduler.IRequestScheduler;
 import org.apache.cassandra.scheduler.NoScheduler;
+import org.apache.cassandra.security.EncryptionContext;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.thrift.ThriftServer;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -92,6 +93,7 @@ public class DatabaseDescriptor
 
     private static String localDC;
     private static Comparator<InetAddress> localComparator;
+    private static EncryptionContext encryptionContext;
 
     public static void forceStaticInitialization() {}
     static
@@ -613,6 +615,10 @@ public class DatabaseDescriptor
 
         if (conf.user_defined_function_fail_timeout < conf.user_defined_function_warn_timeout)
             throw new ConfigurationException("user_defined_function_warn_timeout must less than user_defined_function_fail_timeout", false);
+
+        // always attempt to load the cipher factory, as we could be in the situation where the user has disabled encryption,
+        // but has existing commitlogs and sstables on disk that are still encrypted (and still need to be read)
+        encryptionContext = new EncryptionContext(config.transparent_data_encryption_options);
     }
 
     private static IEndpointSnitch createEndpointSnitch(String snitchClassName) throws ConfigurationException
@@ -1792,4 +1798,15 @@ public class DatabaseDescriptor
     {
         conf.user_function_timeout_policy = userFunctionTimeoutPolicy;
     }
+
+    public static EncryptionContext getEncryptionContext()
+    {
+        return encryptionContext;
+    }
+
+    @VisibleForTesting
+    public static void setEncryptionContext(EncryptionContext ec)
+    {
+        encryptionContext = ec;
+    } 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/src/java/org/apache/cassandra/config/TransparentDataEncryptionOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/TransparentDataEncryptionOptions.java b/src/java/org/apache/cassandra/config/TransparentDataEncryptionOptions.java
new file mode 100644
index 0000000..4ad0305
--- /dev/null
+++ b/src/java/org/apache/cassandra/config/TransparentDataEncryptionOptions.java
@@ -0,0 +1,76 @@
+/*
+ * 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.cassandra.config;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+
+public class TransparentDataEncryptionOptions
+{
+    public boolean enabled = false;
+    public int chunk_length_kb = 64;
+    public String cipher = "AES/CBC/PKCS5Padding";
+    public String key_alias;
+    public int iv_length = 16;
+
+    public ParameterizedClass key_provider;
+
+    public TransparentDataEncryptionOptions()
+    {   }
+
+    public TransparentDataEncryptionOptions(boolean enabled)
+    {
+        this.enabled = enabled;
+    }
+
+    public TransparentDataEncryptionOptions(String cipher, String keyAlias, ParameterizedClass keyProvider)
+    {
+        this(true, cipher, keyAlias, keyProvider);
+    }
+
+    public TransparentDataEncryptionOptions(boolean enabled, String cipher, String keyAlias, ParameterizedClass keyProvider)
+    {
+        this.enabled = enabled;
+        this.cipher = cipher;
+        key_alias = keyAlias;
+        key_provider = keyProvider;
+    }
+
+    public String get(String key)
+    {
+        return key_provider.parameters.get(key);
+    }
+
+    @VisibleForTesting
+    public void remove(String key)
+    {
+        key_provider.parameters.remove(key);
+    }
+
+    public boolean equals(Object o)
+    {
+        return o instanceof TransparentDataEncryptionOptions && equals((TransparentDataEncryptionOptions) o);
+    }
+
+    public boolean equals(TransparentDataEncryptionOptions other)
+    {
+        // not sure if this is a great equals() impl....
+        return Objects.equal(cipher, other.cipher) &&
+               Objects.equal(key_alias, other.key_alias);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/src/java/org/apache/cassandra/security/CipherFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/CipherFactory.java b/src/java/org/apache/cassandra/security/CipherFactory.java
new file mode 100644
index 0000000..0ff9867
--- /dev/null
+++ b/src/java/org/apache/cassandra/security/CipherFactory.java
@@ -0,0 +1,175 @@
+/*
+ * 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.cassandra.security;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.Key;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.util.Arrays;
+import java.util.concurrent.ExecutionException;
+import javax.crypto.Cipher;
+import javax.crypto.NoSuchPaddingException;
+import javax.crypto.spec.IvParameterSpec;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.TransparentDataEncryptionOptions;
+
+/**
+ * A factory for loading encryption keys from {@link KeyProvider} instances.
+ * Maintains a cache of loaded keys to avoid invoking the key provider on every call.
+ */
+public class CipherFactory
+{
+    private final Logger logger = LoggerFactory.getLogger(CipherFactory.class);
+
+    /**
+     * Keep around thread local instances of Cipher as they are quite expensive to instantiate (@code Cipher#getInstance).
+     * Bonus points if you can avoid calling (@code Cipher#init); hence, the point of the supporting struct
+     * for caching Cipher instances.
+     */
+    private static final ThreadLocal<CachedCipher> cipherThreadLocal = new ThreadLocal<>();
+
+    private final SecureRandom secureRandom;
+    private final LoadingCache<String, Key> cache;
+    private final int ivLength;
+    private final KeyProvider keyProvider;
+
+    public CipherFactory(TransparentDataEncryptionOptions options)
+    {
+        logger.info("initializing CipherFactory");
+        ivLength = options.iv_length;
+
+        try
+        {
+            secureRandom = SecureRandom.getInstance("SHA1PRNG");
+            Class<KeyProvider> keyProviderClass = (Class<KeyProvider>)Class.forName(options.key_provider.class_name);
+            Constructor ctor = keyProviderClass.getConstructor(TransparentDataEncryptionOptions.class);
+            keyProvider = (KeyProvider)ctor.newInstance(options);
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException("couldn't load cipher factory", e);
+        }
+
+        cache = CacheBuilder.newBuilder() // by default cache is unbounded
+                .maximumSize(64) // a value large enough that we should never even get close (so nothing gets evicted)
+                .concurrencyLevel(Runtime.getRuntime().availableProcessors())
+                .removalListener(new RemovalListener<String, Key>()
+                {
+                    public void onRemoval(RemovalNotification<String, Key> notice)
+                    {
+                        // maybe reload the key? (to avoid the reload being on the user's dime)
+                        logger.info("key {} removed from cipher key cache", notice.getKey());
+                    }
+                })
+                .build(new CacheLoader<String, Key>()
+                {
+                    @Override
+                    public Key load(String alias) throws Exception
+                    {
+                        logger.info("loading secret key for alias {}", alias);
+                        return keyProvider.getSecretKey(alias);
+                    }
+                });
+    }
+
+    public Cipher getEncryptor(String transformation, String keyAlias) throws IOException
+    {
+        byte[] iv = new byte[ivLength];
+        secureRandom.nextBytes(iv);
+        return buildCipher(transformation, keyAlias, iv, Cipher.ENCRYPT_MODE);
+    }
+
+    public Cipher getDecryptor(String transformation, String keyAlias, byte[] iv) throws IOException
+    {
+        assert iv != null || iv.length > 0 : "trying to decrypt, but the initialization vector is empty";
+        return buildCipher(transformation, keyAlias, iv, Cipher.DECRYPT_MODE);
+    }
+
+    @VisibleForTesting
+    Cipher buildCipher(String transformation, String keyAlias, byte[] iv, int cipherMode) throws IOException
+    {
+        try
+        {
+            CachedCipher cachedCipher = cipherThreadLocal.get();
+            if (cachedCipher != null)
+            {
+                Cipher cipher = cachedCipher.cipher;
+                // rigorous checks to make sure we've absolutely got the correct instance (with correct alg/key/iv/...)
+                if (cachedCipher.mode == cipherMode && cipher.getAlgorithm().equals(transformation)
+                    && cachedCipher.keyAlias.equals(keyAlias) && Arrays.equals(cipher.getIV(), iv))
+                    return cipher;
+            }
+
+            Key key = retrieveKey(keyAlias);
+            Cipher cipher = Cipher.getInstance(transformation);
+            cipher.init(cipherMode, key, new IvParameterSpec(iv));
+            cipherThreadLocal.set(new CachedCipher(cipherMode, keyAlias, cipher));
+            return cipher;
+        }
+        catch (NoSuchAlgorithmException | NoSuchPaddingException | InvalidAlgorithmParameterException | InvalidKeyException e)
+        {
+            logger.error("could not build cipher", e);
+            throw new IOException("cannot load cipher", e);
+        }
+    }
+
+    private Key retrieveKey(String keyAlias) throws IOException
+    {
+        try
+        {
+            return cache.get(keyAlias);
+        }
+        catch (ExecutionException e)
+        {
+            if (e.getCause() instanceof IOException)
+                throw (IOException)e.getCause();
+            throw new IOException("failed to load key from cache: " + keyAlias, e);
+        }
+    }
+
+    /**
+     * A simple struct to use with the thread local caching of Cipher as we can't get the mode (encrypt/decrypt) nor
+     * key_alias (or key!) from the Cipher itself to use for comparisons
+     */
+    private static class CachedCipher
+    {
+        public final int mode;
+        public final String keyAlias;
+        public final Cipher cipher;
+
+        private CachedCipher(int mode, String keyAlias, Cipher cipher)
+        {
+            this.mode = mode;
+            this.keyAlias = keyAlias;
+            this.cipher = cipher;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/src/java/org/apache/cassandra/security/EncryptionContext.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/EncryptionContext.java b/src/java/org/apache/cassandra/security/EncryptionContext.java
new file mode 100644
index 0000000..dff6894
--- /dev/null
+++ b/src/java/org/apache/cassandra/security/EncryptionContext.java
@@ -0,0 +1,122 @@
+/*
+ * 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.cassandra.security;
+
+import java.io.IOException;
+import java.util.Collections;
+import javax.crypto.Cipher;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+
+import org.apache.cassandra.config.TransparentDataEncryptionOptions;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+
+/**
+ * A (largely) immutable wrapper for the application-wide file-level encryption settings.
+ */
+public class EncryptionContext
+{
+    public static final String ENCRYPTION_CIPHER = "encCipher";
+    public static final String ENCRYPTION_KEY_ALIAS = "encKeyAlias";
+    public static final String ENCRYPTION_IV = "encIV";
+
+    private final TransparentDataEncryptionOptions tdeOptions;
+    private final ICompressor compressor;
+    private final CipherFactory cipherFactory;
+
+    private final int chunkLength;
+
+    public EncryptionContext()
+    {
+        this(new TransparentDataEncryptionOptions());
+    }
+
+    public EncryptionContext(TransparentDataEncryptionOptions tdeOptions)
+    {
+        this(tdeOptions, true);
+    }
+
+    @VisibleForTesting
+    public EncryptionContext(TransparentDataEncryptionOptions tdeOptions, boolean init)
+    {
+        this.tdeOptions = tdeOptions;
+        compressor = LZ4Compressor.create(Collections.<String, String>emptyMap());
+        chunkLength = tdeOptions.chunk_length_kb * 1024;
+
+        // always attempt to load the cipher factory, as we could be in the situation where the user has disabled encryption,
+        // but has existing commitlogs and sstables on disk that are still git addencrypted (and still need to be read)
+        CipherFactory factory = null;
+
+        if (tdeOptions.enabled && init)
+        {
+            try
+            {
+                factory = new CipherFactory(tdeOptions);
+            }
+            catch (Exception e)
+            {
+                throw new ConfigurationException("failed to load key provider for transparent data encryption", e);
+            }
+        }
+
+        cipherFactory = factory;
+    }
+
+    public ICompressor getCompressor()
+    {
+        return compressor;
+    }
+
+    public Cipher getEncryptor() throws IOException
+    {
+        return cipherFactory.getEncryptor(tdeOptions.cipher, tdeOptions.key_alias);
+    }
+
+    public Cipher getDecryptor(byte[] IV) throws IOException
+    {
+        return cipherFactory.getDecryptor(tdeOptions.cipher, tdeOptions.key_alias, IV);
+    }
+
+    public boolean isEnabled()
+    {
+        return tdeOptions.enabled;
+    }
+
+    public int getChunkLength()
+    {
+        return chunkLength;
+    }
+
+    public TransparentDataEncryptionOptions getTransparentDataEncryptionOptions()
+    {
+        return tdeOptions;
+    }
+
+    public boolean equals(Object o)
+    {
+        return o instanceof EncryptionContext && equals((EncryptionContext) o);
+    }
+
+    public boolean equals(EncryptionContext other)
+    {
+        return Objects.equal(tdeOptions, other.tdeOptions) && Objects.equal(compressor, other.compressor);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/src/java/org/apache/cassandra/security/JKSKeyProvider.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/JKSKeyProvider.java b/src/java/org/apache/cassandra/security/JKSKeyProvider.java
new file mode 100644
index 0000000..8d7f1c6
--- /dev/null
+++ b/src/java/org/apache/cassandra/security/JKSKeyProvider.java
@@ -0,0 +1,90 @@
+/*
+ * 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.cassandra.security;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.security.Key;
+import java.security.KeyStore;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.TransparentDataEncryptionOptions;
+import org.apache.cassandra.io.util.FileUtils;
+
+/**
+ * A {@code KeyProvider} that retrieves keys from a java keystore.
+ */
+public class JKSKeyProvider implements KeyProvider
+{
+    private final Logger logger = LoggerFactory.getLogger(JKSKeyProvider.class);
+    static final String PROP_KEYSTORE = "keystore";
+    static final String PROP_KEYSTORE_PW = "keystore_password";
+    static final String PROP_KEYSTORE_TYPE = "store_type";
+    static final String PROP_KEY_PW = "key_password";
+
+    private final KeyStore store;
+    private final boolean isJceks;
+    private final TransparentDataEncryptionOptions options;
+
+    public JKSKeyProvider(TransparentDataEncryptionOptions options)
+    {
+        this.options = options;
+        logger.info("initializing keystore from file {}", options.get(PROP_KEYSTORE));
+        FileInputStream inputStream = null;
+        try
+        {
+            inputStream = new FileInputStream(options.get(PROP_KEYSTORE));
+            store = KeyStore.getInstance(options.get(PROP_KEYSTORE_TYPE));
+            store.load(inputStream, options.get(PROP_KEYSTORE_PW).toCharArray());
+            isJceks = store.getType().equalsIgnoreCase("jceks");
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException("couldn't load keystore", e);
+        }
+        finally
+        {
+            FileUtils.closeQuietly(inputStream);
+        }
+    }
+
+    public Key getSecretKey(String keyAlias) throws IOException
+    {
+        // there's a lovely behavior with jceks files that all aliases are lower-cased
+        if (isJceks)
+            keyAlias = keyAlias.toLowerCase();
+
+        Key key;
+        try
+        {
+            String password = options.get(PROP_KEY_PW);
+            if (password == null || password.isEmpty())
+                password = options.get(PROP_KEYSTORE_PW);
+            key = store.getKey(keyAlias, password.toCharArray());
+        }
+        catch (Exception e)
+        {
+            throw new IOException("unable to load key from keystore");
+        }
+        if (key == null)
+            throw new IOException(String.format("key %s was not found in keystore", keyAlias));
+        return key;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/src/java/org/apache/cassandra/security/KeyProvider.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/security/KeyProvider.java b/src/java/org/apache/cassandra/security/KeyProvider.java
new file mode 100644
index 0000000..f380aed
--- /dev/null
+++ b/src/java/org/apache/cassandra/security/KeyProvider.java
@@ -0,0 +1,33 @@
+/*
+ * 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.cassandra.security;
+
+import java.io.IOException;
+import java.security.Key;
+
+/**
+ * Customizable key retrieval mechanism. Implementations should expect that retrieved keys will be cached.
+ * Further, each key will be requested non-concurrently (that is, no stampeding herds for the same key), although
+ * unique keys may be requested concurrently (unless you mark {@code getSecretKey} synchronized).
+ *
+ * Implementations must provide a constructor that accepts {@code TransparentDataEncryptionOptions} as the sole parameter.
+ */
+public interface KeyProvider
+{
+    Key getSecretKey(String alias) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/test/conf/cassandra.keystore
----------------------------------------------------------------------
diff --git a/test/conf/cassandra.keystore b/test/conf/cassandra.keystore
new file mode 100644
index 0000000..9a704ca
Binary files /dev/null and b/test/conf/cassandra.keystore differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/test/conf/cassandra_encryption.yaml
----------------------------------------------------------------------
diff --git a/test/conf/cassandra_encryption.yaml b/test/conf/cassandra_encryption.yaml
new file mode 100644
index 0000000..47e1312
--- /dev/null
+++ b/test/conf/cassandra_encryption.yaml
@@ -0,0 +1,14 @@
+transparent_data_encryption_options:
+    enabled: true
+    chunk_length_kb: 2
+    cipher: AES/CBC/PKCS5Padding
+    key_alias: testing:1
+    # CBC requires iv length to be 16 bytes
+    # iv_length: 16
+    key_provider: 
+      - class_name: org.apache.cassandra.security.JKSKeyProvider
+        parameters: 
+          - keystore: test/conf/cassandra.keystore
+            keystore_password: cassandra
+            store_type: JCEKS
+            key_password: cassandra

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/test/unit/org/apache/cassandra/security/CipherFactoryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/security/CipherFactoryTest.java b/test/unit/org/apache/cassandra/security/CipherFactoryTest.java
new file mode 100644
index 0000000..4239973
--- /dev/null
+++ b/test/unit/org/apache/cassandra/security/CipherFactoryTest.java
@@ -0,0 +1,87 @@
+package org.apache.cassandra.security;
+
+import java.io.IOException;
+import java.security.SecureRandom;
+
+import javax.crypto.BadPaddingException;
+import javax.crypto.Cipher;
+import javax.crypto.IllegalBlockSizeException;
+
+import com.google.common.base.Charsets;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.TransparentDataEncryptionOptions;
+
+public class CipherFactoryTest
+{
+    // http://www.gutenberg.org/files/4300/4300-h/4300-h.htm
+    static final String ULYSSEUS = "Stately, plump Buck Mulligan came from the stairhead, bearing a bowl of lather on which a mirror and a razor lay crossed. " +
+                                   "A yellow dressinggown, ungirdled, was sustained gently behind him on the mild morning air. He held the bowl aloft and intoned: " +
+                                   "—Introibo ad altare Dei.";
+    TransparentDataEncryptionOptions encryptionOptions;
+    CipherFactory cipherFactory;
+    SecureRandom secureRandom;
+
+    @Before
+    public void setup()
+    {
+        secureRandom = new SecureRandom(new byte[] {0,1,2,3,4,5,6,7,8,9} );
+        encryptionOptions = EncryptionContextGenerator.createEncryptionOptions();
+        cipherFactory = new CipherFactory(encryptionOptions);
+    }
+
+    @Test
+    public void roundTrip() throws IOException, BadPaddingException, IllegalBlockSizeException
+    {
+        Cipher encryptor = cipherFactory.getEncryptor(encryptionOptions.cipher, encryptionOptions.key_alias);
+        byte[] original = ULYSSEUS.getBytes(Charsets.UTF_8);
+        byte[] encrypted = encryptor.doFinal(original);
+
+        Cipher decryptor = cipherFactory.getDecryptor(encryptionOptions.cipher, encryptionOptions.key_alias, encryptor.getIV());
+        byte[] decrypted = decryptor.doFinal(encrypted);
+        Assert.assertEquals(ULYSSEUS, new String(decrypted, Charsets.UTF_8));
+    }
+
+    private byte[] nextIV()
+    {
+        byte[] b = new byte[16];
+        secureRandom.nextBytes(b);
+        return b;
+    }
+
+    @Test
+    public void buildCipher_SameParams() throws Exception
+    {
+        byte[] iv = nextIV();
+        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.ENCRYPT_MODE);
+        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.ENCRYPT_MODE);
+        Assert.assertTrue(c1 == c2);
+    }
+
+    @Test
+    public void buildCipher_DifferentModes() throws Exception
+    {
+        byte[] iv = nextIV();
+        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.ENCRYPT_MODE);
+        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, iv, Cipher.DECRYPT_MODE);
+        Assert.assertFalse(c1 == c2);
+    }
+
+    @Test
+    public void buildCipher_DifferentIVs() throws Exception
+    {
+        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, nextIV(), Cipher.ENCRYPT_MODE);
+        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, nextIV(), Cipher.DECRYPT_MODE);
+        Assert.assertFalse(c1 == c2);
+    }
+
+    @Test
+    public void buildCipher_DifferentAliases() throws Exception
+    {
+        Cipher c1 = cipherFactory.buildCipher(encryptionOptions.cipher, encryptionOptions.key_alias, nextIV(), Cipher.ENCRYPT_MODE);
+        Cipher c2 = cipherFactory.buildCipher(encryptionOptions.cipher, EncryptionContextGenerator.KEY_ALIAS_2, nextIV(), Cipher.DECRYPT_MODE);
+        Assert.assertFalse(c1 == c2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/test/unit/org/apache/cassandra/security/EncryptionContextGenerator.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/security/EncryptionContextGenerator.java b/test/unit/org/apache/cassandra/security/EncryptionContextGenerator.java
new file mode 100644
index 0000000..635889b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/security/EncryptionContextGenerator.java
@@ -0,0 +1,54 @@
+/*
+ *
+ * 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.cassandra.security;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.config.TransparentDataEncryptionOptions;
+
+public class EncryptionContextGenerator
+{
+    public static final String KEY_ALIAS_1 = "testing:1";
+    public static final String KEY_ALIAS_2 = "testing:2";
+
+    public static EncryptionContext createContext(boolean init)
+    {
+        return new EncryptionContext(createEncryptionOptions(), init);
+    }
+
+    public static TransparentDataEncryptionOptions createEncryptionOptions()
+    {
+        Map<String,String> params = new HashMap<>();
+        params.put("keystore", "test/conf/cassandra.keystore");
+        params.put("keystore_password", "cassandra");
+        params.put("store_type", "JCEKS");
+        ParameterizedClass keyProvider = new ParameterizedClass(JKSKeyProvider.class.getName(), params);
+
+        return new TransparentDataEncryptionOptions("AES/CBC/PKCS5Padding", KEY_ALIAS_1, keyProvider);
+    }
+
+    public static EncryptionContext createDisabledContext()
+    {
+        return new EncryptionContext();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/765ab3fc/test/unit/org/apache/cassandra/security/JKSKeyProviderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/security/JKSKeyProviderTest.java b/test/unit/org/apache/cassandra/security/JKSKeyProviderTest.java
new file mode 100644
index 0000000..081f688
--- /dev/null
+++ b/test/unit/org/apache/cassandra/security/JKSKeyProviderTest.java
@@ -0,0 +1,52 @@
+/*
+ * 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.cassandra.security;
+
+import java.io.IOException;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.config.TransparentDataEncryptionOptions;
+
+public class JKSKeyProviderTest
+{
+    JKSKeyProvider jksKeyProvider;
+    TransparentDataEncryptionOptions tdeOptions;
+
+    @Before
+    public void setup()
+    {
+        tdeOptions = EncryptionContextGenerator.createEncryptionOptions();
+        jksKeyProvider = new JKSKeyProvider(tdeOptions);
+    }
+
+    @Test
+    public void getSecretKey_WithKeyPassword() throws IOException
+    {
+        Assert.assertNotNull(jksKeyProvider.getSecretKey(tdeOptions.key_alias));
+    }
+
+    @Test
+    public void getSecretKey_WithoutKeyPassword() throws IOException
+    {
+        tdeOptions.remove("key_password");
+        Assert.assertNotNull(jksKeyProvider.getSecretKey(tdeOptions.key_alias));
+    }
+}