Updated Branches: refs/heads/trunk f2960cd49 -> cc499166b
FLUME-1578. Proposal to modify file channel encryption config. (Brock Noland via Mike Percy) Project: http://git-wip-us.apache.org/repos/asf/flume/repo Commit: http://git-wip-us.apache.org/repos/asf/flume/commit/cc499166 Tree: http://git-wip-us.apache.org/repos/asf/flume/tree/cc499166 Diff: http://git-wip-us.apache.org/repos/asf/flume/diff/cc499166 Branch: refs/heads/trunk Commit: cc499166bc958bd11191a2d40d9d0cfde27a7bdc Parents: f2960cd Author: Mike Percy <[email protected]> Authored: Thu Sep 13 14:21:45 2012 -0700 Committer: Mike Percy <[email protected]> Committed: Thu Sep 13 14:28:02 2012 -0700 ---------------------------------------------------------------------- .../channel/file/EventQueueBackingStoreFile.java | 11 ++++- .../org/apache/flume/channel/file/FileChannel.java | 32 ++++++++------ .../file/encryption/EncryptionConfiguration.java | 6 +-- .../file/encryption/KeyProviderFactory.java | 4 +- .../file/TestEventQueueBackingStoreFactory.java | 32 +++++++++++++++ .../file/encryption/EncryptionTestUtils.java | 18 +++----- .../file/encryption/TestFileChannelEncryption.java | 32 +++++++-------- .../file/encryption/TestJCEFileKeyProvider.java | 16 +++----- 8 files changed, 89 insertions(+), 62 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java index eab9e4a..5eaf8c2 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/EventQueueBackingStoreFile.java @@ -62,9 +62,8 @@ abstract class EventQueueBackingStoreFile extends EventQueueBackingStore { super(capacity, name); this.checkpointFile = checkpointFile; checkpointFileHandle = new RandomAccessFile(checkpointFile, "rw"); - + int totalBytes = (capacity + HEADER_SIZE) * Serialization.SIZE_OF_LONG; if(checkpointFileHandle.length() == 0) { - int totalBytes = (capacity + HEADER_SIZE) * Serialization.SIZE_OF_LONG; allocate(checkpointFile, totalBytes); checkpointFileHandle.seek(INDEX_VERSION * Serialization.SIZE_OF_LONG); checkpointFileHandle.writeLong(getVersion()); @@ -72,6 +71,14 @@ abstract class EventQueueBackingStoreFile extends EventQueueBackingStore { LOG.info("Preallocated " + checkpointFile + " to " + checkpointFileHandle.length() + " for capacity " + capacity); } + if(checkpointFile.length() != totalBytes) { + String msg = "Configured capacity is " + capacity + " but the " + + " checkpoint file capacity is " + + ((checkpointFile.length() / Serialization.SIZE_OF_LONG) - HEADER_SIZE) + + ". See FileChannel documentation on how to change a channels" + + " capacity."; + throw new IllegalStateException(msg); + } mappedBuffer = checkpointFileHandle.getChannel().map(MapMode.READ_WRITE, 0, checkpointFile.length()); elementsBuffer = mappedBuffer.asLongBuffer(); http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java index 0312cfe..c2e904b 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/FileChannel.java @@ -40,7 +40,6 @@ import org.apache.flume.instrumentation.ChannelCounter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.base.Strings; @@ -94,7 +93,7 @@ public class FileChannel extends BasicChannelSemantics { private boolean useLogReplayV1; private boolean useFastReplay = false; private KeyProvider encryptionKeyProvider; - private String encryptionKeyAlias; + private String encryptionActiveKey; private String encryptionCipherProvider; @Override @@ -212,25 +211,30 @@ public class FileChannel extends BasicChannelSemantics { ".")); String encryptionKeyProviderName = encryptionContext.getString( EncryptionConfiguration.KEY_PROVIDER); - encryptionKeyAlias = encryptionContext.getString( - EncryptionConfiguration.KEY_ALIAS); + encryptionActiveKey = encryptionContext.getString( + EncryptionConfiguration.ACTIVE_KEY); encryptionCipherProvider = encryptionContext.getString( EncryptionConfiguration.CIPHER_PROVIDER); if(encryptionKeyProviderName != null) { - Preconditions.checkState(!Strings.isNullOrEmpty(encryptionKeyAlias), - "encryptionKeyAlias"); + Preconditions.checkState(!Strings.isNullOrEmpty(encryptionActiveKey), + "Encryption configuration problem: " + + EncryptionConfiguration.ACTIVE_KEY + " is missing"); Preconditions.checkState(!Strings.isNullOrEmpty(encryptionCipherProvider), - "encryptionCipherProvider"); + "Encryption configuration problem: " + + EncryptionConfiguration.CIPHER_PROVIDER + " is missing"); Context keyProviderContext = new Context(encryptionContext. - getSubProperties(Joiner.on("."). - join(EncryptionConfiguration.KEY_PROVIDER, - encryptionKeyProviderName.trim(), ""))); + getSubProperties(EncryptionConfiguration.KEY_PROVIDER + ".")); encryptionKeyProvider = KeyProviderFactory. - getInstance(keyProviderContext); + getInstance(encryptionKeyProviderName, keyProviderContext); } else { - Preconditions.checkState(encryptionKeyAlias == null, "encryptionKeyAlias"); + Preconditions.checkState(encryptionActiveKey == null, + "Encryption configuration problem: " + + EncryptionConfiguration.ACTIVE_KEY + " is present while key " + + "provider name is not."); Preconditions.checkState(encryptionCipherProvider == null, - "encryptionCipherProvider"); + "Encryption configuration problem: " + + EncryptionConfiguration.CIPHER_PROVIDER + " is present while " + + "key provider name is not."); } if(queueRemaining == null) { @@ -262,7 +266,7 @@ public class FileChannel extends BasicChannelSemantics { builder.setUseLogReplayV1(useLogReplayV1); builder.setUseFastReplay(useFastReplay); builder.setEncryptionKeyProvider(encryptionKeyProvider); - builder.setEncryptionKeyAlias(encryptionKeyAlias); + builder.setEncryptionKeyAlias(encryptionActiveKey); builder.setEncryptionCipherProvider(encryptionCipherProvider); log = builder.build(); log.replay(); http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/EncryptionConfiguration.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/EncryptionConfiguration.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/EncryptionConfiguration.java index 27716fc..aaea0cd 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/EncryptionConfiguration.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/EncryptionConfiguration.java @@ -28,13 +28,9 @@ public class EncryptionConfiguration { */ public static final String KEY_PROVIDER = "keyProvider"; /** - * Encryption key provider type, default is null. - */ - public static final String KEY_PROVIDER_TYPE = "type"; - /** * Encryption key alias, default is null. */ - public static final String KEY_ALIAS = "keyAlias"; + public static final String ACTIVE_KEY = "activeKey"; /** * Encryption cipher provider, default is null. */ http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java index f09c48b..60d51ed 100644 --- a/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java +++ b/flume-ng-channels/flume-file-channel/src/main/java/org/apache/flume/channel/file/encryption/KeyProviderFactory.java @@ -30,9 +30,7 @@ public class KeyProviderFactory { LoggerFactory.getLogger(KeyProviderFactory.class); @SuppressWarnings({ "rawtypes", "unchecked" }) - public static KeyProvider getInstance(Context context) { - String keyProviderType = context.getString( - EncryptionConfiguration.KEY_PROVIDER_TYPE); + public static KeyProvider getInstance(String keyProviderType, Context context) { Preconditions.checkNotNull(keyProviderType, "key provider type must not be null"); http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java index 1fc9b49..b1a55be 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/TestEventQueueBackingStoreFactory.java @@ -82,6 +82,38 @@ public class TestEventQueueBackingStoreFactory { Serialization.VERSION_2, pointersInTestCheckpoint); } @Test + public void testDecreaseCapacity() throws Exception { + Assert.assertTrue(checkpoint.delete()); + EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. + get(checkpoint, 10, "test"); + backingStore.close(); + try { + EventQueueBackingStoreFactory.get(checkpoint, 9, "test"); + Assert.fail(); + } catch (IllegalStateException e) { + String expected = "Configured capacity is 9 but the checkpoint file " + + "capacity is 10. See FileChannel documentation on how to change " + + "a channels capacity."; + Assert.assertEquals(expected, e.getMessage()); + } + } + @Test + public void testIncreaseCapacity() throws Exception { + Assert.assertTrue(checkpoint.delete()); + EventQueueBackingStore backingStore = EventQueueBackingStoreFactory. + get(checkpoint, 10, "test"); + backingStore.close(); + try { + EventQueueBackingStoreFactory.get(checkpoint, 11, "test"); + Assert.fail(); + } catch (IllegalStateException e) { + String expected = "Configured capacity is 11 but the checkpoint file " + + "capacity is 10. See FileChannel documentation on how to change " + + "a channels capacity."; + Assert.assertEquals(expected, e.getMessage()); + } + } + @Test public void testNewCheckpoint() throws Exception { Assert.assertTrue(checkpoint.delete()); verify(EventQueueBackingStoreFactory.get(checkpoint, 10, "test", false), http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java index fc617f5..8fbf2b1 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/EncryptionTestUtils.java @@ -95,9 +95,9 @@ public class EncryptionTestUtils { result.put("key-1", null); return result; } - public static Map<String,String> configureForKeyStore(String keyProviderName, - File keyStoreFile, File keyStorePasswordFile, - Map<String, File> keyAliasPassword) throws Exception { + public static Map<String,String> configureForKeyStore(File keyStoreFile, + File keyStorePasswordFile, Map<String, File> keyAliasPassword) + throws Exception { Map<String, String> context = Maps.newHashMap(); List<String> keys = Lists.newArrayList(); Joiner joiner = Joiner.on("."); @@ -107,26 +107,22 @@ public class EncryptionTestUtils { keys.add(alias); } else { String propertyName = joiner.join(EncryptionConfiguration.KEY_PROVIDER, - keyProviderName, EncryptionConfiguration.JCE_FILE_KEYS, alias, + EncryptionConfiguration.JCE_FILE_KEYS, alias, EncryptionConfiguration.JCE_FILE_KEY_PASSWORD_FILE); keys.add(alias); context.put(propertyName, passwordFile.getAbsolutePath()); } } - context.put(EncryptionConfiguration.KEY_PROVIDER, keyProviderName); - context.put(joiner.join(EncryptionConfiguration.KEY_PROVIDER, - keyProviderName, EncryptionConfiguration.KEY_PROVIDER_TYPE), - KeyProviderType.JCEKSFILE.name()); context.put(joiner.join(EncryptionConfiguration.KEY_PROVIDER, - keyProviderName, EncryptionConfiguration.JCE_FILE_KEY_STORE_FILE), + EncryptionConfiguration.JCE_FILE_KEY_STORE_FILE), keyStoreFile.getAbsolutePath()); if(keyStorePasswordFile != null) { context.put(joiner.join(EncryptionConfiguration.KEY_PROVIDER, - keyProviderName, EncryptionConfiguration.JCE_FILE_KEY_STORE_PASSWORD_FILE), + EncryptionConfiguration.JCE_FILE_KEY_STORE_PASSWORD_FILE), keyStorePasswordFile.getAbsolutePath()); } context.put(joiner.join(EncryptionConfiguration.KEY_PROVIDER, - keyProviderName, EncryptionConfiguration.JCE_FILE_KEYS), + EncryptionConfiguration.JCE_FILE_KEYS), Joiner.on(" ").join(keys)); return context; } http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java index a0037b8..d2f5208 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestFileChannelEncryption.java @@ -50,7 +50,6 @@ import com.google.common.io.Files; public class TestFileChannelEncryption extends TestFileChannelBase { protected static final Logger LOGGER = LoggerFactory.getLogger(TestFileChannelEncryption.class); - private static final String KEY_PROVIDER_NAME = "myKeyProvider"; private File keyStoreFile; private File keyStorePasswordFile; private Map<String, File> keyAliasPassword; @@ -78,11 +77,13 @@ public class TestFileChannelEncryption extends TestFileChannelBase { private Map<String, String> getOverridesForEncryption() throws Exception { Map<String, String> overrides = getOverrides(); Map<String, String> encryptionProps = EncryptionTestUtils. - configureForKeyStore(KEY_PROVIDER_NAME, keyStoreFile, + configureForKeyStore(keyStoreFile, keyStorePasswordFile, keyAliasPassword); + encryptionProps.put(EncryptionConfiguration.KEY_PROVIDER, + KeyProviderType.JCEKSFILE.name()); encryptionProps.put(EncryptionConfiguration.CIPHER_PROVIDER, CipherProviderType.AESCTRNOPADDING.name()); - encryptionProps.put(EncryptionConfiguration.KEY_ALIAS, "key-1"); + encryptionProps.put(EncryptionConfiguration.ACTIVE_KEY, "key-1"); for(String key : encryptionProps.keySet()) { overrides.put(EncryptionConfiguration.ENCRYPTION_PREFIX + "." + key, encryptionProps.get(key)); @@ -166,16 +167,15 @@ public class TestFileChannelEncryption extends TestFileChannelBase { @Test public void testConfiguration() throws Exception { Map<String, String> overrides = Maps.newHashMap(); - overrides.put("encryption.keyAlias", "key-1"); + overrides.put("encryption.activeKey", "key-1"); overrides.put("encryption.cipherProvider", "AESCTRNOPADDING"); - overrides.put("encryption.keyProvider", "myKeyProvider"); - overrides.put("encryption.keyProvider.myKeyProvider.type", "JCEKSFILE"); - overrides.put("encryption.keyProvider.myKeyProvider.keyStoreFile", + overrides.put("encryption.keyProvider", "JCEKSFILE"); + overrides.put("encryption.keyProvider.keyStoreFile", keyStoreFile.getAbsolutePath()); - overrides.put("encryption.keyProvider.myKeyProvider.keyStorePasswordFile", + overrides.put("encryption.keyProvider.keyStorePasswordFile", keyStorePasswordFile.getAbsolutePath()); - overrides.put("encryption.keyProvider.myKeyProvider.keys", "key-0 key-1"); - overrides.put("encryption.keyProvider.myKeyProvider.keys.key-0.passwordFile", + overrides.put("encryption.keyProvider.keys", "key-0 key-1"); + overrides.put("encryption.keyProvider.keys.key-0.passwordFile", keyAliasPassword.get("key-0").getAbsolutePath()); channel = createFileChannel(overrides); channel.start(); @@ -250,8 +250,7 @@ public class TestFileChannelEncryption extends TestFileChannelBase { public void testBadKeyProviderInvalidValue() throws Exception { Map<String, String> overrides = getOverridesForEncryption(); overrides.put(Joiner.on(".").join(EncryptionConfiguration.ENCRYPTION_PREFIX, - EncryptionConfiguration.KEY_PROVIDER, KEY_PROVIDER_NAME, - EncryptionConfiguration.KEY_PROVIDER_TYPE), "invalid"); + EncryptionConfiguration.KEY_PROVIDER), "invalid"); try { channel = createFileChannel(overrides); Assert.fail(); @@ -264,8 +263,7 @@ public class TestFileChannelEncryption extends TestFileChannelBase { public void testBadKeyProviderInvalidClass() throws Exception { Map<String, String> overrides = getOverridesForEncryption(); overrides.put(Joiner.on(".").join(EncryptionConfiguration.ENCRYPTION_PREFIX, - EncryptionConfiguration.KEY_PROVIDER, KEY_PROVIDER_NAME, - EncryptionConfiguration.KEY_PROVIDER_TYPE), String.class.getName()); + EncryptionConfiguration.KEY_PROVIDER), String.class.getName()); try { channel = createFileChannel(overrides); Assert.fail(); @@ -296,7 +294,7 @@ public class TestFileChannelEncryption extends TestFileChannelBase { public void testMissingKeyStoreFile() throws Exception { Map<String, String> overrides = getOverridesForEncryption(); overrides.put(Joiner.on(".").join(EncryptionConfiguration.ENCRYPTION_PREFIX, - EncryptionConfiguration.KEY_PROVIDER, KEY_PROVIDER_NAME, + EncryptionConfiguration.KEY_PROVIDER, EncryptionConfiguration.JCE_FILE_KEY_STORE_FILE), "/path/does/not/exist"); try { @@ -311,7 +309,7 @@ public class TestFileChannelEncryption extends TestFileChannelBase { public void testMissingKeyStorePasswordFile() throws Exception { Map<String, String> overrides = getOverridesForEncryption(); overrides.put(Joiner.on(".").join(EncryptionConfiguration.ENCRYPTION_PREFIX, - EncryptionConfiguration.KEY_PROVIDER, KEY_PROVIDER_NAME, + EncryptionConfiguration.KEY_PROVIDER, EncryptionConfiguration.JCE_FILE_KEY_STORE_PASSWORD_FILE), "/path/does/not/exist"); try { @@ -339,7 +337,7 @@ public class TestFileChannelEncryption extends TestFileChannelBase { public void testBadKeyAlias() throws Exception { Map<String, String> overrides = getOverridesForEncryption(); overrides.put(EncryptionConfiguration.ENCRYPTION_PREFIX + "." + - EncryptionConfiguration.KEY_ALIAS, "invalid"); + EncryptionConfiguration.ACTIVE_KEY, "invalid"); channel = TestUtils.createFileChannel(checkpointDir.getAbsolutePath(), dataDir, overrides); channel.start(); http://git-wip-us.apache.org/repos/asf/flume/blob/cc499166/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java ---------------------------------------------------------------------- diff --git a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java index 14afaeb..f33cada 100644 --- a/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java +++ b/flume-ng-channels/flume-file-channel/src/test/java/org/apache/flume/channel/file/encryption/TestJCEFileKeyProvider.java @@ -32,12 +32,10 @@ import org.junit.Before; import org.junit.Test; import com.google.common.base.Charsets; -import com.google.common.base.Joiner; import com.google.common.collect.Maps; import com.google.common.io.Files; public class TestJCEFileKeyProvider { - private static final String KEY_PROVIDER_NAME = "myKeyProvider"; private CipherProvider.Encryptor encryptor; private CipherProvider.Decryptor decryptor; private File baseDir; @@ -72,13 +70,12 @@ public class TestJCEFileKeyProvider { EncryptionTestUtils.createKeyStore(keyStoreFile, keyStorePasswordFile, keyAliasPassword); Context context = new Context(EncryptionTestUtils. - configureForKeyStore(KEY_PROVIDER_NAME, keyStoreFile, + configureForKeyStore(keyStoreFile, keyStorePasswordFile, keyAliasPassword)); Context keyProviderContext = new Context( - context.getSubProperties(Joiner.on(".").join( - EncryptionConfiguration.KEY_PROVIDER, KEY_PROVIDER_NAME, ""))); + context.getSubProperties(EncryptionConfiguration.KEY_PROVIDER + ".")); KeyProvider keyProvider = KeyProviderFactory. - getInstance(keyProviderContext); + getInstance(KeyProviderType.JCEKSFILE.name(), keyProviderContext); testKeyProvider(keyProvider); } @Test @@ -86,13 +83,12 @@ public class TestJCEFileKeyProvider { keyAliasPassword.putAll(EncryptionTestUtils. configureTestKeyStore(baseDir, keyStoreFile)); Context context = new Context(EncryptionTestUtils. - configureForKeyStore(KEY_PROVIDER_NAME, keyStoreFile, + configureForKeyStore(keyStoreFile, keyStorePasswordFile, keyAliasPassword)); Context keyProviderContext = new Context( - context.getSubProperties(Joiner.on(".").join( - EncryptionConfiguration.KEY_PROVIDER, KEY_PROVIDER_NAME, ""))); + context.getSubProperties(EncryptionConfiguration.KEY_PROVIDER + ".")); KeyProvider keyProvider = KeyProviderFactory. - getInstance(keyProviderContext); + getInstance(KeyProviderType.JCEKSFILE.name(), keyProviderContext); testKeyProvider(keyProvider); } private void createNewKeyStore() throws Exception {
