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/01/26 21:17:29 UTC

[5/5] incubator-nifi git commit: NIFI-250: Implemented fingerprinting and reloading of controller services and reporting tasks from flow.xml

NIFI-250: Implemented fingerprinting and reloading of controller services and reporting tasks from flow.xml


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

Branch: refs/heads/NIFI-250
Commit: 2df4500c05ac91d46614e9ca41974fed890d2646
Parents: 86d15f9
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Jan 26 15:17:04 2015 -0500
Committer: Mark Payne <ma...@hotmail.com>
Committed: Mon Jan 26 15:17:04 2015 -0500

----------------------------------------------------------------------
 .../cluster/manager/impl/WebClusterManager.java |  8 +-
 .../service/ControllerServiceProvider.java      |  2 +-
 .../apache/nifi/controller/FlowController.java  | 55 ++++++++++++-
 .../controller/StandardFlowSynchronizer.java    | 70 ++++++++++++++++-
 .../service/ControllerServiceLoader.java        |  3 +-
 .../StandardControllerServiceProvider.java      |  6 +-
 .../nifi/fingerprint/FingerprintFactory.java    | 81 ++++++++++++++++++++
 .../nifi/web/controller/ControllerFacade.java   |  8 +-
 8 files changed, 219 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/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 54c2b55..9d9640d 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
@@ -1287,6 +1287,10 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
             writeLock.unlock("handleControllerStartupFailure");
         }
     }
+    
+    public ControllerServiceNode createControllerService(final String type, final boolean firstTimeAdded) {
+    	return createControllerService(type, UUID.randomUUID().toString(), firstTimeAdded);
+    }
 
     /**
      * Adds an instance of a specified controller service.
@@ -1297,8 +1301,8 @@ public class WebClusterManager implements HttpClusterManager, ProtocolHandler, C
      * @return
      */
     @Override
-    public ControllerServiceNode createControllerService(final String type, final boolean firstTimeAdded) {
-        return controllerServiceProvider.createControllerService(type, firstTimeAdded);
+    public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) {
+        return controllerServiceProvider.createControllerService(type, id, firstTimeAdded);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
index 60ff6c9..1bc3964 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java
@@ -34,7 +34,7 @@ public interface ControllerServiceProvider extends ControllerServiceLookup {
      * @param firstTimeAdded
      * @return
      */
-    ControllerServiceNode createControllerService(String type, boolean firstTimeAdded);
+    ControllerServiceNode createControllerService(String type, String id, boolean firstTimeAdded);
 
     /**
      * Gets the controller service node for the specified identifier. Returns

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/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 92975ad..47e26c0 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
@@ -2589,9 +2589,60 @@ public class FlowController implements EventAccess, ControllerServiceProvider, H
     	}
     }
     
-    @Override
+    
+    /**
+     * <p>
+     * Starts any enabled Processors and Reporting Tasks that are referencing this Controller Service. If other Controller
+     * Services reference this Controller Service, will also enable those services and 'active' any components referencing
+     * them.
+     * </p>
+     * 
+     * <p>
+     * NOTE: If any component cannot be started, an IllegalStateException will be thrown an no more components will
+     * be activated. This method provides no atomicity.
+     * </p>
+     * 
+     * @param serviceNode
+     */
+    public void activateReferencingComponents(final ControllerServiceNode serviceNode) {
+    	final ControllerServiceReference ref = serviceNode.getReferences();
+    	final Set<ConfiguredComponent> components = ref.getReferencingComponents();
+    	
+    	// First, activate any other controller services. We do this first so that we can
+    	// avoid the situation where Processor X depends on Controller Services Y and Z; and
+    	// Controller Service Y depends on Controller Service Z. In this case, if we first attempted
+    	// to start Processor X, we would fail because Controller Service Y is disabled. THis way, we
+    	// can recursively enable everything.
+    	for ( final ConfiguredComponent component : components ) {
+    		if (component instanceof ControllerServiceNode) {
+    			final ControllerServiceNode componentNode = (ControllerServiceNode) component;
+    			enableControllerService(componentNode);
+    			activateReferencingComponents(componentNode);
+    		}
+    	}
+    	
+    	for ( final ConfiguredComponent component : components ) {
+    		if (component instanceof ProcessorNode) {
+    			final ProcessorNode procNode = (ProcessorNode) component;
+    			if ( !procNode.isRunning() ) {
+    				startProcessor(procNode.getProcessGroup().getIdentifier(), procNode.getIdentifier());
+    			}
+    		} else if (component instanceof ReportingTaskNode) {
+    			final ReportingTaskNode taskNode = (ReportingTaskNode) component;
+    			if ( !taskNode.isRunning() ) {
+    				startReportingTask(taskNode);
+    			}
+    		}
+    	}
+    }
+    
     public ControllerServiceNode createControllerService(final String type, final boolean firstTimeAdded) {
-        return controllerServiceProvider.createControllerService(type, firstTimeAdded);
+    	return createControllerService(type, UUID.randomUUID().toString(), firstTimeAdded);
+    }
+    
+    @Override
+    public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) {
+        return controllerServiceProvider.createControllerService(type, id, firstTimeAdded);
     }
     
     public void enableReportingTask(final ReportingTaskNode reportingTaskNode) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
index 0964a91..dbf9b2c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/StandardFlowSynchronizer.java
@@ -238,7 +238,11 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
 	            if ( controllerServicesElement != null ) {
 	                final List<Element> serviceElements = DomUtils.getChildElementsByTagName(controllerServicesElement, "controllerService");
 	                for ( final Element serviceElement : serviceElements ) {
-	                	addControllerService(controller, serviceElement, encryptor);
+	                	if ( !initialized || existingFlowEmpty ) {
+	                		addControllerService(controller, serviceElement, encryptor);
+	                	} else {
+	                		updateControllerService(controller, serviceElement, encryptor);
+	                	}
 	                }
                 }
                 
@@ -246,7 +250,11 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
                 if ( reportingTasksElement != null ) {
                 	final List<Element> taskElements = DomUtils.getChildElementsByTagName(reportingTasksElement, "reportingTask");
                 	for ( final Element taskElement : taskElements ) {
-                		addReportingTask(controller, taskElement, encryptor);
+                		if ( !initialized || existingFlowEmpty ) {
+                			addReportingTask(controller, taskElement, encryptor);
+                		} else {
+                			updateReportingTask(controller, taskElement, encryptor);
+                		}
                 	}
                 }
             }
@@ -347,6 +355,21 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
     	for ( final Map.Entry<String, String> property : dto.getProperties().entrySet() ) {
     		node.setProperty(property.getKey(), property.getValue());
     	}
+    	
+    	if ( dto.getEnabled() == Boolean.TRUE ) {
+    		controller.enableControllerService(node);
+    	}
+    }
+    
+    private void updateControllerService(final FlowController controller, final Element controllerServiceElement, final StringEncryptor encryptor) {
+    	final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElement, encryptor);
+    	
+    	final boolean enabled = controller.isControllerServiceEnabled(dto.getId());
+    	if (dto.getEnabled() && !enabled) {
+    		controller.enableControllerService(controller.getControllerServiceNode(dto.getId()));
+    	} else if (dto.getEnabled() == Boolean.FALSE && enabled) {
+    		controller.disableControllerService(controller.getControllerServiceNode(dto.getId()));
+    	}
     }
     
     private void addReportingTask(final FlowController controller, final Element reportingTaskElement, final StringEncryptor encryptor) throws ReportingTaskInstantiationException {
@@ -367,6 +390,49 @@ public class StandardFlowSynchronizer implements FlowSynchronizer {
     	}
     }
 
+    private void updateReportingTask(final FlowController controller, final Element reportingTaskElement, final StringEncryptor encryptor) {
+    	final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(reportingTaskElement, encryptor);
+    	final ReportingTaskNode taskNode = controller.getReportingTaskNode(dto.getId());
+    	
+        if (!taskNode.getScheduledState().name().equals(dto.getScheduledState())) {
+            try {
+                switch (ScheduledState.valueOf(dto.getScheduledState())) {
+                    case DISABLED:
+                    	if ( taskNode.isRunning() ) {
+                    		controller.stopReportingTask(taskNode);
+                    	}
+                    	controller.disableReportingTask(taskNode);
+                        break;
+                    case RUNNING:
+                    	if ( taskNode.getScheduledState() == ScheduledState.DISABLED ) {
+                    		controller.enableReportingTask(taskNode);
+                    	}
+                    	controller.startReportingTask(taskNode);
+                        break;
+                    case STOPPED:
+                        if (taskNode.getScheduledState() == ScheduledState.DISABLED) {
+                        	controller.enableReportingTask(taskNode);
+                        } else if (taskNode.getScheduledState() == ScheduledState.RUNNING) {
+                        	controller.stopReportingTask(taskNode);
+                        }
+                        break;
+                }
+            } catch (final IllegalStateException ise) {
+                logger.error("Failed to change Scheduled State of {} from {} to {} due to {}", taskNode, taskNode.getScheduledState().name(), dto.getScheduledState(), ise.toString());
+                logger.error("", ise);
+
+                // create bulletin for the Processor Node
+                controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Node Reconnection", Severity.ERROR.name(),
+                        "Failed to change Scheduled State of " + taskNode + " from " + taskNode.getScheduledState().name() + " to " + dto.getScheduledState() + " due to " + ise.toString()));
+
+                // create bulletin at Controller level.
+                controller.getBulletinRepository().addBulletin(BulletinFactory.createBulletin("Node Reconnection", Severity.ERROR.name(),
+                        "Failed to change Scheduled State of " + taskNode + " from " + taskNode.getScheduledState().name() + " to " + dto.getScheduledState() + " due to " + ise.toString()));
+            }
+        }
+    }
+    
+    
     private ProcessGroup updateProcessGroup(final FlowController controller, final ProcessGroup parentGroup, final Element processGroupElement, final StringEncryptor encryptor) throws ProcessorInstantiationException {
 
         // get the parent group ID

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
index 7504d13..db44b5f 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/ControllerServiceLoader.java
@@ -122,10 +122,11 @@ public class ControllerServiceLoader {
                 for (final Element serviceElement : serviceNodes) {
                     //get properties for the specific controller task - id, name, class,
                     //and schedulingPeriod must be set
+                    final String serviceId = DomUtils.getChild(serviceElement, "identifier").getTextContent().trim();
                     final String serviceClass = DomUtils.getChild(serviceElement, "class").getTextContent().trim();
 
                     //set the class to be used for the configured controller task
-                    final ControllerServiceNode serviceNode = provider.createControllerService(serviceClass, false);
+                    final ControllerServiceNode serviceNode = provider.createControllerService(serviceClass, serviceId, false);
 
                     //optional task-specific properties
                     for (final Element optionalProperty : DomUtils.getChildElementsByTagName(serviceElement, "property")) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/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 75de4b9..0263ee0 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;
 
@@ -98,12 +97,11 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     }
 
     @Override
-    public ControllerServiceNode createControllerService(final String type, final boolean firstTimeAdded) {
-        if (type == null) {
+    public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) {
+        if (type == null || id == null) {
             throw new NullPointerException();
         }
         
-        final String id = UUID.randomUUID().toString();
         final ClassLoader currentContextClassLoader = Thread.currentThread().getContextClassLoader();
         try {
             final ClassLoader cl = ExtensionManager.getClassLoader(type);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 8575569..ee7cd54 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -41,12 +41,14 @@ import javax.xml.validation.SchemaFactory;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.FlowFromDOMFactory;
 import org.apache.nifi.controller.Template;
 import org.apache.nifi.controller.exception.ProcessorInstantiationException;
 import org.apache.nifi.encrypt.StringEncryptor;
 import org.apache.nifi.processor.Processor;
 import org.apache.nifi.util.DomUtils;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
+import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.FlowSnippetDTO;
 import org.apache.nifi.web.api.dto.FunnelDTO;
 import org.apache.nifi.web.api.dto.LabelDTO;
@@ -58,6 +60,7 @@ import org.apache.nifi.web.api.dto.ProcessorDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
 import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
+import org.apache.nifi.web.api.dto.ReportingTaskDTO;
 import org.apache.nifi.web.api.dto.TemplateDTO;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
@@ -250,6 +253,22 @@ public final class FingerprintFactory {
         // root group
         final Element rootGroupElem = (Element) DomUtils.getChildNodesByTagName(flowControllerElem, "rootGroup").item(0);
         addProcessGroupFingerprint(builder, rootGroupElem, controller);
+        
+        final Element controllerServicesElem = DomUtils.getChild(flowControllerElem, "controllerServices");
+        if ( controllerServicesElem != null ) {
+        	for ( final Element serviceElem : DomUtils.getChildElementsByTagName(controllerServicesElem, "controllerService") ) {
+        		addControllerServiceFingerprint(builder, serviceElem);
+        	}
+        }
+        
+        final Element reportingTasksElem = DomUtils.getChild(flowControllerElem, "reportingTasks");
+        if ( reportingTasksElem != null ) {
+        	for ( final Element taskElem : DomUtils.getChildElementsByTagName(reportingTasksElem, "reportingTask") ) {
+        		addReportingTaskFingerprint(builder, taskElem);
+        	}
+        }
+        
+        
         return builder;
     }
 
@@ -832,6 +851,68 @@ public final class FingerprintFactory {
         builder.append(funnel.getId());
         return builder;
     }
+    
+    private void addControllerServiceFingerprint(final StringBuilder builder, final Element controllerServiceElem) {
+    	final ControllerServiceDTO dto = FlowFromDOMFactory.getControllerService(controllerServiceElem, encryptor);
+    	addControllerServiceFingerprint(builder, dto);
+    }
+    
+    private void addControllerServiceFingerprint(final StringBuilder builder, final ControllerServiceDTO dto) {
+    	builder.append(dto.getId());
+    	builder.append(dto.getType());
+    	builder.append(dto.getName());
+    	builder.append(dto.getComment());
+    	builder.append(dto.getAvailability());
+    	builder.append(dto.getAnnotationData());
+    	
+    	final Map<String, String> properties = dto.getProperties();
+    	if (properties == null) {
+            builder.append("NO_PROPERTIES");
+        } else {
+            final SortedMap<String, String> sortedProps = new TreeMap<>(properties);
+            for (final Map.Entry<String, String> entry : sortedProps.entrySet()) {
+                final String propName = entry.getKey();
+                final String propValue = entry.getValue();
+                if (propValue == null) {
+                    continue;
+                }
+
+                builder.append(propName).append("=").append(propValue);
+            }
+        }
+    }
+    
+    private void addReportingTaskFingerprint(final StringBuilder builder, final Element element) {
+    	final ReportingTaskDTO dto = FlowFromDOMFactory.getReportingTask(element, encryptor);
+    	addReportingTaskFingerprint(builder, dto);
+    }
+    
+    private void addReportingTaskFingerprint(final StringBuilder builder, final ReportingTaskDTO dto) {
+    	builder.append(dto.getId());
+    	builder.append(dto.getType());
+    	builder.append(dto.getName());
+    	builder.append(dto.getComment());
+    	builder.append(dto.getSchedulingPeriod());
+    	builder.append(dto.getSchedulingStrategy());
+    	builder.append(dto.getAvailability());
+    	builder.append(dto.getAnnotationData());
+    	
+    	final Map<String, String> properties = dto.getProperties();
+    	if (properties == null) {
+            builder.append("NO_PROPERTIES");
+        } else {
+            final SortedMap<String, String> sortedProps = new TreeMap<>(properties);
+            for (final Map.Entry<String, String> entry : sortedProps.entrySet()) {
+                final String propName = entry.getKey();
+                final String propValue = entry.getValue();
+                if (propValue == null) {
+                    continue;
+                }
+
+                builder.append(propName).append("=").append(propValue);
+            }
+        }
+    }
 
     private Comparator<Element> getIdsComparator() {
         return new Comparator<Element>() {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/2df4500c/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index 5ed25e6..40d3b1b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -404,9 +404,13 @@ public class ControllerFacade implements ControllerServiceProvider {
         return flowController.getControllerService(serviceIdentifier);
     }
 
-    @Override
     public ControllerServiceNode createControllerService(final String type, final boolean firstTimeAdded) {
-        return flowController.createControllerService(type, firstTimeAdded);
+    	return flowController.createControllerService(type, firstTimeAdded);
+    }
+    
+    @Override
+    public ControllerServiceNode createControllerService(final String type, final String id, final boolean firstTimeAdded) {
+        return flowController.createControllerService(type, id, firstTimeAdded);
     }
     
     public void removeControllerService(ControllerServiceNode serviceNode) {