You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2015/02/10 17:54:26 UTC

[1/3] incubator-nifi git commit: NIFI-250: Remove references to Controller Service when the referencing component is removed

Repository: incubator-nifi
Updated Branches:
  refs/heads/NIFI-250 ea8cb5984 -> 5390c7626


NIFI-250: Remove references to Controller Service when the referencing component is removed


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/33f55163
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/33f55163
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/33f55163

Branch: refs/heads/NIFI-250
Commit: 33f551630ac2664cf938834c7e450d04da7ec9f0
Parents: ea8cb59
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 10 11:08:05 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 10 11:08:05 2015 -0500

----------------------------------------------------------------------
 .../cluster/manager/impl/WebClusterManager.java | 13 ++++++++++
 .../apache/nifi/controller/FlowController.java  | 14 +++++++++++
 .../StandardControllerServiceProvider.java      | 15 +++++++++++-
 .../nifi/groups/StandardProcessGroup.java       | 25 ++++++++++++++++----
 4 files changed, 61 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/33f55163/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
index 4d768c6..f06c513 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
@@ -1473,6 +1473,19 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
 	            ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext());
 	        }
 	        
+	        for ( final Map.Entry<PropertyDescriptor, String> entry : reportingTaskNode.getProperties().entrySet() ) {
+	            final PropertyDescriptor descriptor = entry.getKey();
+	            if (descriptor.getControllerServiceDefinition() != null ) {
+	                final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue();
+	                if ( value != null ) {
+	                    final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value);
+	                    if ( serviceNode != null ) {
+	                        serviceNode.removeReference(reportingTaskNode);
+	                    }
+	                }
+	            }
+	        }
+	        
 	        reportingTasks.remove(reportingTaskNode.getIdentifier());
     	} finally {
     		writeLock.unlock("removeReportingTask");

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/33f55163/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 71d0012..8bd1149 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -62,6 +62,7 @@ import org.apache.nifi.cluster.protocol.NodeProtocolSender;
 import org.apache.nifi.cluster.protocol.UnknownServiceAddressException;
 import org.apache.nifi.cluster.protocol.message.HeartbeatMessage;
 import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage;
+import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
@@ -2550,6 +2551,19 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, reportingTaskNode.getReportingTask(), reportingTaskNode.getConfigurationContext());
         }
         
+        for ( final Map.Entry<PropertyDescriptor, String> entry : reportingTaskNode.getProperties().entrySet() ) {
+            final PropertyDescriptor descriptor = entry.getKey();
+            if (descriptor.getControllerServiceDefinition() != null ) {
+                final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue();
+                if ( value != null ) {
+                    final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value);
+                    if ( serviceNode != null ) {
+                        serviceNode.removeReference(reportingTaskNode);
+                    }
+                }
+            }
+        }
+        
         reportingTasks.remove(reportingTaskNode.getIdentifier());
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/33f55163/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index 5eed713..208da30 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -28,7 +28,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -36,6 +35,7 @@ import org.apache.nifi.annotation.lifecycle.OnAdded;
 import org.apache.nifi.annotation.lifecycle.OnDisabled;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
+import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.controller.ControllerService;
 import org.apache.nifi.controller.ValidationContextFactory;
@@ -243,6 +243,19 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
             ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnRemoved.class, serviceNode.getControllerServiceImplementation(), configurationContext);
         }
         
+        for ( final Map.Entry<PropertyDescriptor, String> entry : serviceNode.getProperties().entrySet() ) {
+            final PropertyDescriptor descriptor = entry.getKey();
+            if (descriptor.getControllerServiceDefinition() != null ) {
+                final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue();
+                if ( value != null ) {
+                    final ControllerServiceNode referencedNode = getControllerServiceNode(value);
+                    if ( referencedNode != null ) {
+                        referencedNode.removeReference(serviceNode);
+                    }
+                }
+            }
+        }
+        
         controllerServices.remove(serviceNode.getIdentifier());
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/33f55163/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index 856ccc1..86533af 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -30,8 +30,13 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
 import org.apache.nifi.annotation.lifecycle.OnRemoved;
 import org.apache.nifi.annotation.lifecycle.OnShutdown;
+import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.ConnectableType;
 import org.apache.nifi.connectable.Connection;
@@ -45,7 +50,9 @@ import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.Snippet;
 import org.apache.nifi.controller.exception.ProcessorLifeCycleException;
 import org.apache.nifi.controller.label.Label;
+import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.logging.LogRepositoryFactory;
 import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.processor.StandardProcessContext;
@@ -53,11 +60,6 @@ import org.apache.nifi.remote.RemoteGroupPort;
 import org.apache.nifi.remote.RootGroupPort;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.util.ReflectionUtils;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-import org.apache.nifi.encrypt.StringEncryptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -674,6 +676,19 @@ public final class StandardProcessGroup implements ProcessGroup {
                 throw new ProcessorLifeCycleException("Failed to invoke 'OnRemoved' methods of " + processor, e);
             }
 
+            for ( final Map.Entry<PropertyDescriptor, String> entry : processor.getProperties().entrySet() ) {
+                final PropertyDescriptor descriptor = entry.getKey();
+                if (descriptor.getControllerServiceDefinition() != null ) {
+                    final String value = entry.getValue() == null ? descriptor.getDefaultValue() : entry.getValue();
+                    if ( value != null ) {
+                        final ControllerServiceNode serviceNode = controllerServiceProvider.getControllerServiceNode(value);
+                        if ( serviceNode != null ) {
+                            serviceNode.removeReference(processor);
+                        }
+                    }
+                }
+            }
+            
             processors.remove(id);
             LogRepositoryFactory.getRepository(processor.getIdentifier()).removeAllObservers();
 


[2/3] incubator-nifi git commit: NIFI-250: Persist Controller Services and Reporting Tasks on NCM

Posted by ma...@apache.org.
NIFI-250: Persist Controller Services and Reporting Tasks on NCM


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/c25a2cae
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/c25a2cae
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/c25a2cae

Branch: refs/heads/NIFI-250
Commit: c25a2caecd31638d3564ea4892986d0580234783
Parents: 33f5516
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 10 11:53:51 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 10 11:53:51 2015 -0500

----------------------------------------------------------------------
 .../nifi/cluster/flow/impl/DataFlowDaoImpl.java | 27 +++++++++++---------
 .../cluster/manager/impl/WebClusterManager.java |  4 ++-
 2 files changed, 18 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c25a2cae/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
index 0d7caf3..dd9d2a3 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java
@@ -410,13 +410,7 @@ public class DataFlowDaoImpl implements DataFlowDao {
         final File stateFile = new File(dir, FLOW_PACKAGE);
         stateFile.createNewFile();
 
-        final byte[] flowBytes = getEmptyFlowBytes();
-        final byte[] templateBytes = new byte[0];
-        final byte[] snippetBytes = new byte[0];
-        final DataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes);
-
-        final ClusterMetadata clusterMetadata = new ClusterMetadata();
-        writeDataFlow(stateFile, dataFlow, clusterMetadata);
+        writeDataFlow(stateFile, new ClusterDataFlow(null, null, new byte[0], new byte[0]), new ClusterMetadata());
 
         return stateFile;
     }
@@ -548,7 +542,7 @@ public class DataFlowDaoImpl implements DataFlowDao {
         clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId());
 
         // write to disk
-        writeDataFlow(file, dataFlow, clusterMetadata);
+        writeDataFlow(file, clusterDataFlow, clusterMetadata);
     }
 
     private void writeTarEntry(final TarArchiveOutputStream tarOut, final String filename, final byte[] bytes) throws IOException {
@@ -559,14 +553,23 @@ public class DataFlowDaoImpl implements DataFlowDao {
         tarOut.closeArchiveEntry();
     }
 
-    private void writeDataFlow(final File file, final DataFlow dataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException {
+    private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException {
 
         try (final OutputStream fos = new FileOutputStream(file);
                 final TarArchiveOutputStream tarOut = new TarArchiveOutputStream(new BufferedOutputStream(fos))) {
 
-            writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow());
-            writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates());
-            writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets());
+            final DataFlow dataFlow = clusterDataFlow.getDataFlow();
+            if ( dataFlow == null ) {
+                writeTarEntry(tarOut, FLOW_XML_FILENAME, getEmptyFlowBytes());
+                writeTarEntry(tarOut, TEMPLATES_FILENAME, new byte[0]);
+                writeTarEntry(tarOut, SNIPPETS_FILENAME, new byte[0]);
+            } else {
+                writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow());
+                writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates());
+                writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets());
+            }
+            writeTarEntry(tarOut, CONTROLLER_SERVICES_FILENAME, clusterDataFlow.getControllerServices());
+            writeTarEntry(tarOut, REPORTING_TASKS_FILENAME, clusterDataFlow.getReportingTasks());
 
             final ByteArrayOutputStream baos = new ByteArrayOutputStream(256);
             writeClusterMetadata(clusterMetadata, baos);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/c25a2cae/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
index f06c513..8a431ad 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java
@@ -1378,6 +1378,7 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
         final DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
         final Document document = docBuilder.newDocument();
     	final Element rootElement = document.createElement("controllerServices");
+    	document.appendChild(rootElement);
     	
     	for ( final ControllerServiceNode serviceNode : getAllControllerServices() ) {
     		StandardFlowSerializer.addControllerService(rootElement, serviceNode, encryptor);
@@ -1390,7 +1391,8 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
     	final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
         final DocumentBuilder docBuilder = docFactory.newDocumentBuilder();
         final Document document = docBuilder.newDocument();
-    	final Element rootElement = document.createElement("controllerServices");
+    	final Element rootElement = document.createElement("reportingTasks");
+    	document.appendChild(rootElement);
     	
     	for ( final ReportingTaskNode taskNode : getReportingTasks() ) {
     		StandardFlowSerializer.addReportingTask(rootElement, taskNode, encryptor);


[3/3] incubator-nifi git commit: NIFI-250: Fixed NPE in StandardSSLContextService

Posted by ma...@apache.org.
NIFI-250: Fixed NPE in StandardSSLContextService


Project: http://git-wip-us.apache.org/repos/asf/incubator-nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-nifi/commit/5390c762
Tree: http://git-wip-us.apache.org/repos/asf/incubator-nifi/tree/5390c762
Diff: http://git-wip-us.apache.org/repos/asf/incubator-nifi/diff/5390c762

Branch: refs/heads/NIFI-250
Commit: 5390c7626a99c076daf05d06118815f902ab7b51
Parents: c25a2ca
Author: Mark Payne <ma...@hotmail.com>
Authored: Tue Feb 10 11:54:17 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Tue Feb 10 11:54:17 2015 -0500

----------------------------------------------------------------------
 .../nifi/ssl/StandardSSLContextService.java     | 37 ++++++++++++++++++--
 1 file changed, 35 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/5390c762/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
index 39bb5fb..34f1844 100644
--- a/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
+++ b/nifi/nifi-nar-bundles/nifi-standard-services/nifi-ssl-context-bundle/nifi-ssl-context-service/src/main/java/org/apache/nifi/ssl/StandardSSLContextService.java
@@ -94,6 +94,7 @@ public class StandardSSLContextService extends AbstractControllerService impleme
             .build();
 
     private static final List<PropertyDescriptor> properties;
+    private ConfigurationContext configContext;
 
     static {
         List<PropertyDescriptor> props = new ArrayList<>();
@@ -105,7 +106,6 @@ public class StandardSSLContextService extends AbstractControllerService impleme
         props.add(TRUSTSTORE_TYPE);
         properties = Collections.unmodifiableList(props);
     }
-    private ConfigurationContext configContext;
 
     @OnEnabled
     public void onConfigured(final ConfigurationContext context) throws InitializationException {
@@ -194,7 +194,7 @@ public class StandardSSLContextService extends AbstractControllerService impleme
         if (results.isEmpty()) {
             // verify that the filename, password, and type match
             try {
-                createSSLContext(ClientAuth.REQUIRED);
+                verifySslConfig(validationContext);
             } catch (ProcessException e) {
                 results.add(new ValidationResult.Builder()
                         .subject(getClass().getSimpleName() + " : " + getIdentifier())
@@ -205,6 +205,39 @@ public class StandardSSLContextService extends AbstractControllerService impleme
         }
         return results;
     }
+    
+    private void verifySslConfig(final ValidationContext validationContext) throws ProcessException {
+        try {
+            final String keystoreFile = validationContext.getProperty(KEYSTORE).getValue();
+            if (keystoreFile == null) {
+                SslContextFactory.createTrustSslContext(
+                        validationContext.getProperty(TRUSTSTORE).getValue(),
+                        validationContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
+                        validationContext.getProperty(TRUSTSTORE_TYPE).getValue());
+                return;
+            }
+            final String truststoreFile = validationContext.getProperty(TRUSTSTORE).getValue();
+            if (truststoreFile == null) {
+                SslContextFactory.createSslContext(
+                        validationContext.getProperty(KEYSTORE).getValue(),
+                        validationContext.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
+                        validationContext.getProperty(KEYSTORE_TYPE).getValue());
+                return;
+            }
+
+            SslContextFactory.createSslContext(
+                    validationContext.getProperty(KEYSTORE).getValue(),
+                    validationContext.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(),
+                    validationContext.getProperty(KEYSTORE_TYPE).getValue(),
+                    validationContext.getProperty(TRUSTSTORE).getValue(),
+                    validationContext.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(),
+                    validationContext.getProperty(TRUSTSTORE_TYPE).getValue(),
+                    org.apache.nifi.security.util.SslContextFactory.ClientAuth.REQUIRED);
+        } catch (final Exception e) {
+            throw new ProcessException(e);
+        }
+    }
+    
 
     @Override
     public SSLContext createSSLContext(final ClientAuth clientAuth) throws ProcessException {