You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2014/12/16 20:05:31 UTC

[03/19] incubator-nifi git commit: NIFI-169 well it finally all builds. There is a classpath issue still to sort out which impacts startup

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java
new file mode 100644
index 0000000..b5f3fd6
--- /dev/null
+++ b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java
@@ -0,0 +1,530 @@
+/*
+ * 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.nifi.distributed.cache.server;
+
+import org.apache.nifi.distributed.cache.server.DistributedSetCacheServer;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.ConnectException;
+import java.nio.charset.StandardCharsets;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService;
+import org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
+import org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+import org.apache.nifi.ssl.StandardSSLContextService;
+import org.apache.nifi.util.MockConfigurationContext;
+import org.apache.nifi.util.MockControllerServiceInitializationContext;
+
+import org.apache.commons.lang3.SerializationException;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestServerAndClient {
+
+    private static Logger LOGGER;
+
+    static {
+        System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
+        System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.server.AbstractCacheServer", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.client.DistributedMapCacheClientService", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.server.TestServerAndClient", "debug");
+        System.setProperty("org.slf4j.simpleLogger.log.nifi.remote.io.socket.ssl.SSLSocketChannel", "trace");
+        LOGGER = LoggerFactory.getLogger(TestServerAndClient.class);
+    }
+
+    @Test
+    public void testNonPersistentSetServerAndClient() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+        final boolean added = client.addIfAbsent("test", serializer);
+        assertTrue(added);
+
+        final boolean contains = client.contains("test", serializer);
+        assertTrue(contains);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean removed = client.remove("test", serializer);
+        assertTrue(removed);
+
+        final boolean containedAfterRemove = client.contains("test", serializer);
+        assertFalse(containedAfterRemove);
+
+        server.shutdownServer();
+    }
+
+    @Test
+    public void testPersistentSetServerAndClient() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final File dataFile = new File("target/cache-data");
+        deleteRecursively(dataFile);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+        final boolean added = client.addIfAbsent("test", serializer);
+        final boolean added2 = client.addIfAbsent("test2", serializer);
+        assertTrue(added);
+        assertTrue(added2);
+
+        final boolean contains = client.contains("test", serializer);
+        final boolean contains2 = client.contains("test2", serializer);
+        assertTrue(contains);
+        assertTrue(contains2);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean removed = client.remove("test", serializer);
+        assertTrue(removed);
+
+        final boolean containedAfterRemove = client.contains("test", serializer);
+        assertFalse(containedAfterRemove);
+
+        server.shutdownServer();
+
+        final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
+        newServer.initialize(newServerInitContext);
+
+        final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
+                newServerInitContext.getControllerServiceLookup());
+        newServer.startServer(newServerContext);
+
+        assertFalse(client.contains("test", serializer));
+        assertTrue(client.contains("test2", serializer));
+
+        newServer.shutdownServer();
+    }
+
+    @Test
+    public void testPersistentSetServerAndClientWithLFUEvictions() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final File dataFile = new File("target/cache-data");
+        deleteRecursively(dataFile);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
+        serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
+
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+        final boolean added = client.addIfAbsent("test", serializer);
+        waitABit();
+        final boolean added2 = client.addIfAbsent("test2", serializer);
+        waitABit();
+        final boolean added3 = client.addIfAbsent("test3", serializer);
+        waitABit();
+        assertTrue(added);
+        assertTrue(added2);
+        assertTrue(added3);
+
+        final boolean contains = client.contains("test", serializer);
+        final boolean contains2 = client.contains("test2", serializer);
+        assertTrue(contains);
+        assertTrue(contains2);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean added4 = client.addIfAbsent("test4", serializer);
+        assertTrue(added4);
+
+        // ensure that added3 was evicted because it was used least frequently
+        assertFalse(client.contains("test3", serializer));
+
+        server.shutdownServer();
+
+        final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
+        newServer.initialize(newServerInitContext);
+
+        final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
+                newServerInitContext.getControllerServiceLookup());
+        newServer.startServer(newServerContext);
+
+        assertTrue(client.contains("test", serializer));
+        assertTrue(client.contains("test2", serializer));
+        assertFalse(client.contains("test3", serializer));
+        assertTrue(client.contains("test4", serializer));
+
+        newServer.shutdownServer();
+    }
+
+    @Test
+    public void testPersistentSetServerAndClientWithFIFOEvictions() throws InitializationException, IOException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedSetCacheServer server = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final File dataFile = new File("target/cache-data");
+        deleteRecursively(dataFile);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath());
+        serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3");
+        serverProperties.put(DistributedSetCacheServer.EVICTION_POLICY, DistributedSetCacheServer.EVICTION_STRATEGY_FIFO);
+
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        final DistributedSetCacheClientService client = createClient();
+        final Serializer<String> serializer = new StringSerializer();
+
+        // add 3 entries to the cache. But, if we add too fast, we'll have the same millisecond
+        // for the entry time so we don't know which entry will be evicted. So we wait a few millis in between
+        final boolean added = client.addIfAbsent("test", serializer);
+        waitABit();
+        final boolean added2 = client.addIfAbsent("test2", serializer);
+        waitABit();
+        final boolean added3 = client.addIfAbsent("test3", serializer);
+        waitABit();
+
+        assertTrue(added);
+        assertTrue(added2);
+        assertTrue(added3);
+
+        final boolean contains = client.contains("test", serializer);
+        final boolean contains2 = client.contains("test2", serializer);
+        assertTrue(contains);
+        assertTrue(contains2);
+
+        final boolean addedAgain = client.addIfAbsent("test", serializer);
+        assertFalse(addedAgain);
+
+        final boolean added4 = client.addIfAbsent("test4", serializer);
+        assertTrue(added4);
+
+        // ensure that added3 was evicted because it was used least frequently
+        assertFalse(client.contains("test", serializer));
+        assertTrue(client.contains("test3", serializer));
+
+        server.shutdownServer();
+
+        final DistributedSetCacheServer newServer = new DistributedSetCacheServer();
+        MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2");
+        newServer.initialize(newServerInitContext);
+
+        final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties,
+                newServerInitContext.getControllerServiceLookup());
+        newServer.startServer(newServerContext);
+
+        assertFalse(client.contains("test", serializer));
+        assertTrue(client.contains("test2", serializer));
+        assertTrue(client.contains("test3", serializer));
+        assertTrue(client.contains("test4", serializer));
+
+        newServer.shutdownServer();
+    }
+
+    @Test
+    public void testNonPersistentMapServerAndClient() throws InitializationException, IOException, InterruptedException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedMapCacheServer server = new DistributedMapCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        DistributedMapCacheClientService client = new DistributedMapCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
+        clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
+        MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.cacheConfig(clientContext);
+        final Serializer<String> valueSerializer = new StringSerializer();
+        final Serializer<String> keySerializer = new StringSerializer();
+        final Deserializer<String> deserializer = new StringDeserializer();
+
+        final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals(null, original);
+        LOGGER.debug("end getAndPutIfAbsent");
+
+        final boolean contains = client.containsKey("testKey", keySerializer);
+        assertTrue(contains);
+        LOGGER.debug("end containsKey");
+
+        final boolean added = client.putIfAbsent("testKey", "test", keySerializer, valueSerializer);
+        assertFalse(added);
+        LOGGER.debug("end putIfAbsent");
+
+        final String originalAfterPut = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals("test", originalAfterPut);
+        LOGGER.debug("end getAndPutIfAbsent");
+
+        final boolean removed = client.remove("testKey", keySerializer);
+        assertTrue(removed);
+        LOGGER.debug("end remove");
+
+        final boolean containedAfterRemove = client.containsKey("testKey", keySerializer);
+        assertFalse(containedAfterRemove);
+
+        client.putIfAbsent("testKey", "test", keySerializer, valueSerializer);
+        client.close();
+        try {
+            client.containsKey("testKey", keySerializer);
+            fail("Should be closed and not accessible");
+        } catch (Exception e) {
+
+        }
+        client = null;
+        clientInitContext = null;
+        clientContext = null;
+
+        DistributedMapCacheClientService client2 = new DistributedMapCacheClientService();
+
+        MockControllerServiceInitializationContext clientInitContext2 = new MockControllerServiceInitializationContext(client2, "client2");
+        client2.initialize(clientInitContext2);
+
+        MockConfigurationContext clientContext2 = new MockConfigurationContext(clientProperties,
+                clientInitContext2.getControllerServiceLookup());
+        client2.cacheConfig(clientContext2);
+        assertFalse(client2.putIfAbsent("testKey", "test", keySerializer, valueSerializer));
+        assertTrue(client2.containsKey("testKey", keySerializer));
+        server.shutdownServer();
+        Thread.sleep(1000);
+        try {
+            client2.containsKey("testKey", keySerializer);
+            fail("Should have blown exception!");
+        } catch (ConnectException e) {
+            client2 = null;
+            clientContext2 = null;
+            clientInitContext2 = null;
+        }
+        Thread.sleep(2000);
+        System.gc();
+        LOGGER.debug("end testNonPersistentMapServerAndClient");
+    }
+
+    @Test
+    public void testClientTermination() throws InitializationException, IOException, InterruptedException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create server
+        final DistributedMapCacheServer server = new DistributedMapCacheServer();
+        MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        DistributedMapCacheClientService client = new DistributedMapCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
+        clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
+        MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.cacheConfig(clientContext);
+        final Serializer<String> valueSerializer = new StringSerializer();
+        final Serializer<String> keySerializer = new StringSerializer();
+        final Deserializer<String> deserializer = new StringDeserializer();
+
+        final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals(null, original);
+
+        final boolean contains = client.containsKey("testKey", keySerializer);
+        assertTrue(contains);
+
+        final boolean added = client.putIfAbsent("testKey", "test", keySerializer, valueSerializer);
+        assertFalse(added);
+
+        final String originalAfterPut = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals("test", originalAfterPut);
+
+        final boolean removed = client.remove("testKey", keySerializer);
+        assertTrue(removed);
+
+        final boolean containedAfterRemove = client.containsKey("testKey", keySerializer);
+        assertFalse(containedAfterRemove);
+
+        client = null;
+        clientInitContext = null;
+        clientContext = null;
+        Thread.sleep(2000);
+        System.gc();
+        server.shutdownServer();
+    }
+
+    @Ignore
+    @Test
+    public void testSSLWith2RequestsWithServerTimeout() throws InitializationException, IOException, InterruptedException {
+        LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
+        // Create SSLContext Service
+        final StandardSSLContextService sslService = new StandardSSLContextService();
+        final MockControllerServiceInitializationContext sslServerInitContext = new MockControllerServiceInitializationContext(sslService,
+                "ssl-context");
+        sslService.initialize(sslServerInitContext);
+
+        final Map<PropertyDescriptor, String> sslServerProps = new HashMap<>();
+        sslServerProps.put(StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks");
+        sslServerProps.put(StandardSSLContextService.KEYSTORE_PASSWORD, "localtest");
+        sslServerProps.put(StandardSSLContextService.KEYSTORE_TYPE, "JKS");
+        sslServerProps.put(StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks");
+        sslServerProps.put(StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest");
+        sslServerProps.put(StandardSSLContextService.TRUSTSTORE_TYPE, "JKS");
+        MockConfigurationContext sslServerContext = new MockConfigurationContext(sslServerProps, sslServerInitContext);
+        sslService.onConfigured(sslServerContext);
+        sslService.createSSLContext(ClientAuth.REQUIRED);
+        // Create server
+        final DistributedMapCacheServer server = new DistributedMapCacheServer();
+        final MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server");
+        server.initialize(serverInitContext);
+
+        final Map<PropertyDescriptor, String> serverProperties = new HashMap<>();
+        serverProperties.put(DistributedMapCacheServer.SSL_CONTEXT_SERVICE, "ssl-context");
+        final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup());
+        server.startServer(serverContext);
+
+        DistributedMapCacheClientService client = new DistributedMapCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost");
+        clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs");
+        clientProperties.put(DistributedMapCacheClientService.SSL_CONTEXT_SERVICE, "ssl-context");
+        MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.cacheConfig(clientContext);
+        final Serializer<String> valueSerializer = new StringSerializer();
+        final Serializer<String> keySerializer = new StringSerializer();
+        final Deserializer<String> deserializer = new StringDeserializer();
+
+        final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer);
+        assertEquals(null, original);
+
+        Thread.sleep(30000);
+        try {
+            final boolean contains = client.containsKey("testKey", keySerializer);
+            assertTrue(contains);
+        } catch (IOException e) {
+            // this is due to the server timing out in the middle of this request
+            assertTrue(e.getMessage().contains("Channel is closed"));
+        }
+
+        server.shutdownServer();
+    }
+
+    private void waitABit() {
+        try {
+            Thread.sleep(10L);
+        } catch (final InterruptedException e) {
+        }
+    }
+
+    private DistributedSetCacheClientService createClient() throws InitializationException {
+        final DistributedSetCacheClientService client = new DistributedSetCacheClientService();
+        MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client");
+        client.initialize(clientInitContext);
+
+        final Map<PropertyDescriptor, String> clientProperties = new HashMap<>();
+        clientProperties.put(DistributedSetCacheClientService.HOSTNAME, "localhost");
+        final MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup());
+        client.onConfigured(clientContext);
+
+        return client;
+    }
+
+    private static class StringSerializer implements Serializer<String> {
+        @Override
+        public void serialize(final String value, final OutputStream output) throws SerializationException, IOException {
+            output.write(value.getBytes(StandardCharsets.UTF_8));
+        }
+    }
+
+    private static class StringDeserializer implements Deserializer<String> {
+        @Override
+        public String deserialize(final byte[] input) throws DeserializationException, IOException {
+            return (input.length == 0) ? null : new String(input, StandardCharsets.UTF_8);
+        }
+    }
+
+    private static void deleteRecursively(final File dataFile) throws IOException {
+        if (dataFile == null || !dataFile.exists()) {
+            return;
+        }
+
+        final File[] children = dataFile.listFiles();
+        for (final File child : children) {
+            if (child.isDirectory()) {
+                deleteRecursively(child);
+            } else {
+                for (int i = 0; i < 100 && child.exists(); i++) {
+                    child.delete();
+                }
+
+                if (child.exists()) {
+                    throw new IOException("Could not delete " + dataFile.getAbsolutePath());
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks
new file mode 100755
index 0000000..81be31d
Binary files /dev/null and b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks
new file mode 100755
index 0000000..820e1e1
Binary files /dev/null and b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml
new file mode 100644
index 0000000..8a34581
--- /dev/null
+++ b/nar-bundles/standard-services/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml
@@ -0,0 +1,47 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>distributed-cache-services-bundle</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>distributed-cache-services-nar</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <name>Distributed Cache Services NAR</name>
+    <packaging>nar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>standard-services-api-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>distributed-cache-client-service</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>distributed-cache-protocol</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>distributed-cache-server</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/distributed-cache-services-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/distributed-cache-services-bundle/pom.xml b/nar-bundles/standard-services/distributed-cache-services-bundle/pom.xml
new file mode 100644
index 0000000..f499831
--- /dev/null
+++ b/nar-bundles/standard-services/distributed-cache-services-bundle/pom.xml
@@ -0,0 +1,35 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>standard-services-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+    
+    <artifactId>distributed-cache-services-bundle</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <name>Distributed Cache Services Bundle</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>distributed-cache-protocol</module>
+        <module>distributed-cache-client-service</module>
+        <module>distributed-cache-server</module>
+        <module>distributed-cache-services-nar</module>
+    </modules>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/load-distribution-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/load-distribution-service-api/pom.xml b/nar-bundles/standard-services/load-distribution-service-api/pom.xml
new file mode 100644
index 0000000..c939e95
--- /dev/null
+++ b/nar-bundles/standard-services/load-distribution-service-api/pom.xml
@@ -0,0 +1,37 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>standard-services-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>load-distribution-service-api</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>jar</packaging>
+
+    <name>Load Distribution Service API</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+    </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionListener.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionListener.java b/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionListener.java
new file mode 100644
index 0000000..656bf99
--- /dev/null
+++ b/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionListener.java
@@ -0,0 +1,24 @@
+/*
+ * 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.nifi.loading;
+
+import java.util.Map;
+
+public interface LoadDistributionListener {
+
+    public void update(Map<String, Integer> loadInfo);
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionService.java b/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionService.java
new file mode 100644
index 0000000..c413975
--- /dev/null
+++ b/nar-bundles/standard-services/load-distribution-service-api/src/main/java/org/apache/nifi/loading/LoadDistributionService.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.nifi.loading;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.controller.ControllerService;
+
+/**
+ * A service that will provide a Map of Fully Qualified Domain Names (fqdn) with
+ * their respective weights (scale of 1 - 100).
+ */
+public interface LoadDistributionService extends ControllerService {
+
+    public Map<String, Integer> getLoadDistribution(Set<String> fqdns);
+
+    public Map<String, Integer> getLoadDistribution(Set<String> fqdns, LoadDistributionListener listener);
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/pom.xml b/nar-bundles/standard-services/pom.xml
new file mode 100644
index 0000000..047d642
--- /dev/null
+++ b/nar-bundles/standard-services/pom.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nar-bundle-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>standard-services-parent</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>pom</packaging>
+
+    <name>Standard Services Parent</name>
+
+    <modules>
+        <module>distributed-cache-client-service-api</module>
+        <module>distributed-cache-services-bundle</module>
+        <module>load-distribution-service-api</module>
+        <module>ssl-context-bundle</module>
+        <module>ssl-context-service-api</module>
+        <module>standard-services-api-nar</module>
+    </modules>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/nar/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/nar/pom.xml b/nar-bundles/standard-services/ssl-context-bundle/nar/pom.xml
new file mode 100644
index 0000000..ec1e937
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/nar/pom.xml
@@ -0,0 +1,39 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>ssl-context-bundle</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+	
+    <artifactId>ssl-context-service-nar</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <name>SSL Context Services Nar</name>
+    <packaging>nar</packaging>
+	
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>standard-services-api-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>ssl-context-service</artifactId>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/pom.xml b/nar-bundles/standard-services/ssl-context-bundle/pom.xml
new file mode 100644
index 0000000..305e242
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/pom.xml
@@ -0,0 +1,34 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>standard-services-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>ssl-context-bundle</artifactId>
+    <version>0.0.1-SNAPSHOT</version>
+    <packaging>pom</packaging>
+
+    <name>SSL Context Bundle</name>
+
+    <modules>
+        <module>ssl-context-service</module>
+        <module>nar</module>
+    </modules>
+    
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/pom.xml b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/pom.xml
new file mode 100644
index 0000000..9e18e57
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/pom.xml
@@ -0,0 +1,52 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>ssl-context-bundle</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>ssl-context-service</artifactId>
+    <packaging>jar</packaging>
+
+    <name>SSL Context Controller Service</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>ssl-context-service-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-processor-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-security-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-mock</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
new file mode 100644
index 0000000..d7aae16
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
@@ -0,0 +1,354 @@
+/*
+ * 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.nifi.ssl;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import javax.net.ssl.SSLContext;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.controller.annotation.OnConfigured;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.security.util.CertificateUtils;
+import org.apache.nifi.security.util.KeystoreType;
+import org.apache.nifi.security.util.SslContextFactory;
+
+public class StandardSSLContextService extends AbstractControllerService implements SSLContextService {
+
+    public static final String STORE_TYPE_JKS = "JKS";
+    public static final String STORE_TYPE_PKCS12 = "PKCS12";
+
+    public static final PropertyDescriptor TRUSTSTORE = new PropertyDescriptor.Builder()
+            .name("Truststore Filename")
+            .description("The fully-qualified filename of the Truststore")
+            .defaultValue(null)
+            .addValidator(createFileExistsAndReadableValidator())
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor TRUSTSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("Truststore Type")
+            .description("The Type of the Truststore. Either JKS or PKCS12")
+            .allowableValues(STORE_TYPE_JKS, STORE_TYPE_PKCS12)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .defaultValue(STORE_TYPE_JKS)
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor TRUSTSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("Truststore Password")
+            .description("The password for the Truststore")
+            .defaultValue(null)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+    public static final PropertyDescriptor KEYSTORE = new PropertyDescriptor.Builder()
+            .name("Keystore Filename")
+            .description("The fully-qualified filename of the Keystore")
+            .defaultValue(null)
+            .addValidator(createFileExistsAndReadableValidator())
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor KEYSTORE_TYPE = new PropertyDescriptor.Builder()
+            .name("Keystore Type")
+            .description("The Type of the Keystore")
+            .allowableValues(STORE_TYPE_JKS, STORE_TYPE_PKCS12)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .defaultValue(STORE_TYPE_JKS)
+            .sensitive(false)
+            .build();
+    public static final PropertyDescriptor KEYSTORE_PASSWORD = new PropertyDescriptor.Builder()
+            .name("Keystore Password")
+            .defaultValue(null)
+            .description("The password for the Keystore")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .sensitive(true)
+            .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(KEYSTORE);
+        props.add(KEYSTORE_PASSWORD);
+        props.add(KEYSTORE_TYPE);
+        props.add(TRUSTSTORE);
+        props.add(TRUSTSTORE_PASSWORD);
+        props.add(TRUSTSTORE_TYPE);
+        properties = Collections.unmodifiableList(props);
+    }
+    private ConfigurationContext configContext;
+
+    @OnConfigured
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        configContext = context;
+
+        final Collection<ValidationResult> results = new ArrayList<>();
+        results.addAll(validateStore(context.getProperties(), KeystoreValidationGroup.KEYSTORE));
+        results.addAll(validateStore(context.getProperties(), KeystoreValidationGroup.TRUSTSTORE));
+
+        if (!results.isEmpty()) {
+            final StringBuilder sb = new StringBuilder(this + " is not valid due to:");
+            for (final ValidationResult result : results) {
+                sb.append("\n").append(result.toString());
+            }
+            throw new InitializationException(sb.toString());
+        }
+
+        if (countNulls(context.getProperty(KEYSTORE).getValue(),
+                context.getProperty(KEYSTORE_PASSWORD).getValue(),
+                context.getProperty(KEYSTORE_TYPE).getValue(),
+                context.getProperty(TRUSTSTORE).getValue(),
+                context.getProperty(TRUSTSTORE_PASSWORD).getValue(),
+                context.getProperty(TRUSTSTORE_TYPE).getValue()) >= 4) {
+            throw new InitializationException(this + " does not have the KeyStore or the TrustStore populated");
+        }
+
+        // verify that the filename, password, and type match
+        createSSLContext(ClientAuth.REQUIRED);
+    }
+
+    private static Validator createFileExistsAndReadableValidator() {
+        return new Validator() {
+            // Not using the FILE_EXISTS_VALIDATOR because the default is to
+            // allow expression language
+            @Override
+            public ValidationResult validate(String subject, String input, ValidationContext context) {
+                final String substituted;
+                try {
+                    substituted = context.newPropertyValue(input).evaluateAttributeExpressions().getValue();
+                } catch (final Exception e) {
+                    return new ValidationResult.Builder()
+                            .subject(subject)
+                            .input(input)
+                            .valid(false)
+                            .explanation("Not a valid Expression Language value: " + e.getMessage())
+                            .build();
+                }
+
+                final File file = new File(substituted);
+                final boolean valid = file.exists() && file.canRead();
+                final String explanation = valid ? null : "File " + file + " does not exist or cannot be read";
+                return new ValidationResult.Builder()
+                        .subject(subject)
+                        .input(input)
+                        .valid(valid)
+                        .explanation(explanation)
+                        .build();
+            }
+        };
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext validationContext) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+        results.addAll(validateStore(validationContext.getProperties(), KeystoreValidationGroup.KEYSTORE));
+        results.addAll(validateStore(validationContext.getProperties(), KeystoreValidationGroup.TRUSTSTORE));
+
+        if (countNulls(validationContext.getProperty(KEYSTORE).getValue(),
+                validationContext.getProperty(KEYSTORE_PASSWORD).getValue(),
+                validationContext.getProperty(KEYSTORE_TYPE).getValue(),
+                validationContext.getProperty(TRUSTSTORE).getValue(),
+                validationContext.getProperty(TRUSTSTORE_PASSWORD).getValue(),
+                validationContext.getProperty(TRUSTSTORE_TYPE).getValue())
+                >= 4) {
+            results.add(new ValidationResult.Builder()
+                    .subject(this.getClass().getSimpleName() + " : " + getIdentifier())
+                    .valid(false)
+                    .explanation("Does not have the KeyStore or the TrustStore populated")
+                    .build());
+        }
+        if (results.isEmpty()) {
+            // verify that the filename, password, and type match
+            try {
+                createSSLContext(ClientAuth.REQUIRED);
+            } catch (ProcessException e) {
+                results.add(new ValidationResult.Builder()
+                        .subject(getClass().getSimpleName() + " : " + getIdentifier())
+                        .valid(false)
+                        .explanation(e.getMessage())
+                        .build());
+            }
+        }
+        return results;
+    }
+
+    @Override
+    public SSLContext createSSLContext(final ClientAuth clientAuth) throws ProcessException {
+        try {
+            final String keystoreFile = configContext.getProperty(KEYSTORE).getValue();
+            if (keystoreFile == null) {
+                return SslContextFactory.createTrustSslContext(
+                        configContext.getProperty(TRUSTSTORE).getValue(),
+                        configContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
+                        configContext.getProperty(TRUSTSTORE_TYPE).getValue());
+            }
+            final String truststoreFile = configContext.getProperty(TRUSTSTORE).getValue();
+            if (truststoreFile == null) {
+                return SslContextFactory.createSslContext(
+                        configContext.getProperty(KEYSTORE).getValue(),
+                        configContext.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
+                        configContext.getProperty(KEYSTORE_TYPE).getValue());
+            }
+
+            return SslContextFactory.createSslContext(
+                    configContext.getProperty(KEYSTORE).getValue(),
+                    configContext.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
+                    configContext.getProperty(KEYSTORE_TYPE).getValue(),
+                    configContext.getProperty(TRUSTSTORE).getValue(),
+                    configContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
+                    configContext.getProperty(TRUSTSTORE_TYPE).getValue(),
+                    org.apache.nifi.security.util.SslContextFactory.ClientAuth.valueOf(clientAuth.name()));
+        } catch (final Exception e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    @Override
+    public String getTrustStoreFile() {
+        return configContext.getProperty(TRUSTSTORE).getValue();
+    }
+
+    @Override
+    public String getTrustStoreType() {
+        return configContext.getProperty(TRUSTSTORE_TYPE).getValue();
+    }
+
+    @Override
+    public String getTrustStorePassword() {
+        return configContext.getProperty(TRUSTSTORE_PASSWORD).getValue();
+    }
+
+    @Override
+    public boolean isTrustStoreConfigured() {
+        return getTrustStoreFile() != null && getTrustStorePassword() != null && getTrustStoreType() != null;
+    }
+
+    @Override
+    public String getKeyStoreFile() {
+        return configContext.getProperty(KEYSTORE).getValue();
+    }
+
+    @Override
+    public String getKeyStoreType() {
+        return configContext.getProperty(KEYSTORE_TYPE).getValue();
+    }
+
+    @Override
+    public String getKeyStorePassword() {
+        return configContext.getProperty(KEYSTORE_PASSWORD).getValue();
+    }
+
+    @Override
+    public boolean isKeyStoreConfigured() {
+        return getKeyStoreFile() != null && getKeyStorePassword() != null && getKeyStoreType() != null;
+    }
+
+    private static Collection<ValidationResult> validateStore(final Map<PropertyDescriptor, String> properties,
+            final KeystoreValidationGroup keyStoreOrTrustStore) {
+        final Collection<ValidationResult> results = new ArrayList<>();
+
+        final String filename;
+        final String password;
+        final String type;
+
+        if (keyStoreOrTrustStore == KeystoreValidationGroup.KEYSTORE) {
+            filename = properties.get(KEYSTORE);
+            password = properties.get(KEYSTORE_PASSWORD);
+            type = properties.get(KEYSTORE_TYPE);
+        } else {
+            filename = properties.get(TRUSTSTORE);
+            password = properties.get(TRUSTSTORE_PASSWORD);
+            type = properties.get(TRUSTSTORE_TYPE);
+        }
+
+        final String keystoreDesc = (keyStoreOrTrustStore == KeystoreValidationGroup.KEYSTORE) ? "Keystore" : "Truststore";
+
+        final int nulls = countNulls(filename, password, type);
+        if (nulls != 3 && nulls != 0) {
+            results.add(new ValidationResult.Builder().valid(false).explanation("Must set either 0 or 3 properties for " + keystoreDesc)
+                    .subject(keystoreDesc + " Properties").build());
+        } else if (nulls == 0) {
+            // all properties were filled in.
+            final File file = new File(filename);
+            if (!file.exists() || !file.canRead()) {
+                results.add(new ValidationResult.Builder()
+                        .valid(false)
+                        .subject(keystoreDesc + " Properties")
+                        .explanation("Cannot access file " + file.getAbsolutePath())
+                        .build());
+            } else {
+                try {
+                    final boolean storeValid = CertificateUtils
+                            .isStoreValid(file.toURI().toURL(), KeystoreType.valueOf(type), password.toCharArray());
+                    if (!storeValid) {
+                        results.add(new ValidationResult.Builder()
+                                .subject(keystoreDesc + " Properties")
+                                .valid(false)
+                                .explanation("Invalid KeyStore Password or Type specified for file " + filename)
+                                .build());
+                    }
+                } catch (MalformedURLException e) {
+                    results.add(new ValidationResult.Builder()
+                            .subject(keystoreDesc + " Properties")
+                            .valid(false)
+                            .explanation("Malformed URL from file: " + e)
+                            .build());
+                }
+            }
+        }
+
+        return results;
+    }
+
+    private static int countNulls(Object... objects) {
+        int count = 0;
+        for (final Object x : objects) {
+            if (x == null) {
+                count++;
+            }
+        }
+
+        return count;
+    }
+
+    public static enum KeystoreValidationGroup {
+
+        KEYSTORE, TRUSTSTORE
+    }
+
+    @Override
+    public String toString() {
+        return "SSLContextService[id=" + getIdentifier() + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
new file mode 100644
index 0000000..b1b6124
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService
@@ -0,0 +1,15 @@
+# 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.
+org.apache.nifi.ssl.StandardSSLContextService
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/index.html
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/index.html b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/index.html
new file mode 100644
index 0000000..0ce9aa4
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/main/resources/docs/org.apache.nifi.ssl.StandardSSLContextService/index.html
@@ -0,0 +1,63 @@
+<!DOCTYPE html>
+<html lang="en">
+    <!--
+      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.
+    -->
+    <head>
+        <meta charset="utf-8" />
+        <title>StandardSSLContextService</title>
+
+        <link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
+    </head>
+
+    <body>
+
+        <!-- Service Documentation ================================================== -->
+        <h2>Configuring the SSLContext Controller Service:</h2>
+        <p>
+            The SSLContext controller service is a mechanism for providing all the security properties that 
+            allow for secure communications between NiFi extensions and other systems. NiFi extensions include 
+            processors, reporting tasks, and other controller services. 
+        </p>
+
+        <p>
+            The <code>controller-services.xml</code> file is located in the NiFi <code>conf</code> 
+            directory. The user may set up any number of controller services within this file.
+        </p>
+
+        <p>
+            Below is an example of the template for a SSLContext controller service. Note that the identifier 
+            in this example is <code>ssl-context</code>. If using this template to create your own SSLContext controller 
+            service, replace the property values with values that are suitable for your system. Possible options for 
+            <code>Keystore Type</code> and <code>Truststore Type</code> are <span style="font-style: italic;">JKS</span>
+            or <span style="font-style: italic;">PKCS12</span>.
+        </p>
+
+        <pre>
+&lt;?xml version="1.0" encoding="UTF-8" ?&gt;
+&lt;services&gt;
+    &lt;service&gt;
+        &lt;identifier&gt;ssl-context&lt;/identifier&gt;
+        &lt;class&gt;nifi.ssl.StandardSSLContextService&lt;/class&gt;
+        &lt;property name="Keystore Filename"&gt;C:/testpki/localtest-ks.jks&lt;/property&gt;
+        &lt;property name="Keystore Password"&gt;localtest&lt;/property&gt;
+        &lt;property name="Keystore Type"&gt;JKS&lt;/property&gt;
+        &lt;property name="Truststore Filename"&gt;C:/testpki/localtest-ts.jks&lt;/property&gt;
+        &lt;property name="Truststore Password"&gt;localtest&lt;/property&gt;
+        &lt;property name="Truststore Type"&gt;JKS&lt;/property&gt;
+    &lt;/service&gt;
+&lt;/services&gt;
+        </pre>
+    </body>
+</html>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
new file mode 100644
index 0000000..5d583ca
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/SSLContextServiceTest.java
@@ -0,0 +1,197 @@
+/*
+ * 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.nifi.ssl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashMap;
+
+import org.apache.nifi.controller.ControllerService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.ssl.SSLContextService.ClientAuth;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SSLContextServiceTest {
+
+    @Test
+    public void testBad1() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            SSLContextService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            runner.addControllerService("test-bad1", service, properties);
+            Assert.fail("Should have thrown an Exception");
+        } catch (InitializationException e) {
+            assertEquals(
+                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad1] does not have the KeyStore or the TrustStore populated",
+                    e.getCause().getCause().toString());
+        }
+    }
+
+    @Test
+    public void testBad2() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            SSLContextService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+            runner.addControllerService("test-bad2", service, properties);
+            Assert.fail("Should have thrown an Exception");
+        } catch (InitializationException e) {
+            assertEquals(
+                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad2] is not valid due to:\n'Keystore Properties' is invalid because Must set either 0 or 3 properties for Keystore",
+                    e.getCause().getCause().toString());
+        }
+    }
+
+    @Test
+    public void testBad3() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            SSLContextService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+            runner.addControllerService("test-bad3", service, properties);
+            Assert.fail("Should have thrown an Exception");
+        } catch (InitializationException e) {
+            assertEquals(
+                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad3] is not valid due to:\n'Truststore Properties' is invalid because Must set either 0 or 3 properties for Truststore",
+                    e.getCause().getCause().toString());
+        }
+    }
+
+    @Test
+    public void testBad4() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            SSLContextService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "wrongpassword");
+            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
+            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+            properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "wrongpassword");
+            properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+            runner.addControllerService("test-bad4", service, properties);
+            Assert.fail("Should have thrown an Exception");
+        } catch (InitializationException e) {
+            assertEquals(
+                    "org.apache.nifi.reporting.InitializationException: SSLContextService[id=test-bad4] is not valid due to:\n"
+                    + "'Keystore Properties' is invalid because Invalid KeyStore Password or Type specified for file src/test/resources/localhost-ks.jks\n"
+                    + "'Truststore Properties' is invalid because Invalid KeyStore Password or Type specified for file src/test/resources/localhost-ts.jks",
+                    e.getCause().getCause().toString());
+        }
+    }
+
+    @Test
+    public void testBad5() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            SSLContextService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/DOES-NOT-EXIST.jks");
+            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
+            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+            properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+            properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+            runner.addControllerService("test-bad5", service, properties);
+            Assert.fail("Should have thrown an Exception");
+        } catch (InitializationException e) {
+            assertTrue(e.getCause().getCause().toString().startsWith("org.apache.nifi.reporting.InitializationException: "
+                    + "SSLContextService[id=test-bad5] is not valid due to:\n'Keystore Properties' is invalid "
+                    + "because Cannot access file"));
+        }
+    }
+
+    @Test
+    public void testGood() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            ControllerService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "PKCS12");
+            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+            properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+            properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+            runner.addControllerService("test-good1", service, properties);
+            runner.setProperty("SSL Context Svc ID", "test-good1");
+            runner.assertValid();
+            service = runner.getProcessContext().getControllerServiceLookup().getControllerService("test-good1");
+            Assert.assertNotNull(service);
+            Assert.assertTrue(service instanceof StandardSSLContextService);
+            SSLContextService sslService = (SSLContextService) service;
+            sslService.createSSLContext(ClientAuth.REQUIRED);
+            sslService.createSSLContext(ClientAuth.WANT);
+            sslService.createSSLContext(ClientAuth.NONE);
+        } catch (InitializationException e) {
+        }
+    }
+
+    @Test
+    public void testGoodTrustOnly() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            SSLContextService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            properties.put(StandardSSLContextService.TRUSTSTORE.getName(), "src/test/resources/localhost-ts.jks");
+            properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "localtest");
+            properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
+            runner.addControllerService("test-good2", service, properties);
+            runner.setProperty("SSL Context Svc ID", "test-good2");
+            runner.assertValid();
+            Assert.assertNotNull(service);
+            Assert.assertTrue(service instanceof StandardSSLContextService);
+            service.createSSLContext(ClientAuth.NONE);
+        } catch (InitializationException e) {
+        }
+    }
+
+    @Test
+    public void testGoodKeyOnly() {
+        try {
+            TestRunner runner = TestRunners.newTestRunner(TestProcessor.class);
+            SSLContextService service = new StandardSSLContextService();
+            HashMap<String, String> properties = new HashMap<String, String>();
+            properties.put(StandardSSLContextService.KEYSTORE.getName(), "src/test/resources/localhost-ks.jks");
+            properties.put(StandardSSLContextService.KEYSTORE_PASSWORD.getName(), "localtest");
+            properties.put(StandardSSLContextService.KEYSTORE_TYPE.getName(), "JKS");
+            runner.addControllerService("test-good3", service, properties);
+            runner.setProperty("SSL Context Svc ID", "test-good3");
+            runner.assertValid();
+            Assert.assertNotNull(service);
+            Assert.assertTrue(service instanceof StandardSSLContextService);
+            SSLContextService sslService = service;
+            sslService.createSSLContext(ClientAuth.NONE);
+        } catch (Exception e) {
+            System.out.println(e);
+            Assert.fail("Should not have thrown a exception " + e.getMessage());
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java
new file mode 100644
index 0000000..6cb2fb6
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/java/org/apache/nifi/ssl/TestProcessor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.nifi.ssl;
+
+import org.apache.nifi.ssl.SSLContextService;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+public class TestProcessor extends AbstractProcessor {
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        List<PropertyDescriptor> propDescs = new ArrayList<>();
+        propDescs.add(new PropertyDescriptor.Builder()
+                .name("SSL Context Svc ID")
+                .description("ID of SSL Context Svc")
+                .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class))
+                .required(true)
+                .build());
+        return propDescs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ks.jks
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ks.jks b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ks.jks
new file mode 100755
index 0000000..81be31d
Binary files /dev/null and b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ks.jks differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ts.jks
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ts.jks b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ts.jks
new file mode 100755
index 0000000..820e1e1
Binary files /dev/null and b/nar-bundles/standard-services/ssl-context-bundle/ssl-context-service/src/test/resources/localhost-ts.jks differ

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/19d4a150/nar-bundles/standard-services/ssl-context-service-api/pom.xml
----------------------------------------------------------------------
diff --git a/nar-bundles/standard-services/ssl-context-service-api/pom.xml b/nar-bundles/standard-services/ssl-context-service-api/pom.xml
new file mode 100644
index 0000000..e71cabb
--- /dev/null
+++ b/nar-bundles/standard-services/ssl-context-service-api/pom.xml
@@ -0,0 +1,36 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      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.
+    -->
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>standard-services-parent</artifactId>
+        <version>0.0.1-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>ssl-context-service-api</artifactId>
+    <packaging>jar</packaging>
+
+    <name>SSL Context Service API</name>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+    </dependencies>
+
+</project>