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/04/10 15:43:53 UTC

[18/62] [abbrv] incubator-nifi git commit: Squashed commit of the following:

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
index a1c4984..c470b99 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceReference.java
@@ -58,23 +58,28 @@ public class StandardControllerServiceReference implements ControllerServiceRefe
     }
 
     @Override
-    public Set<ConfiguredComponent> getRunningReferences() {
-        final Set<ConfiguredComponent> runningReferences = new HashSet<>();
+    public Set<ConfiguredComponent> getActiveReferences() {
+        final Set<ConfiguredComponent> activeReferences = new HashSet<>();
         final Set<ControllerServiceNode> serviceNodes = new HashSet<>();
 
         for (final ConfiguredComponent component : components) {
             if (component instanceof ControllerServiceNode) {
                 serviceNodes.add((ControllerServiceNode) component);
+                
+                final ControllerServiceState state = ((ControllerServiceNode) component).getState();
+                if ( state != ControllerServiceState.DISABLED ) {
+                    activeReferences.add(component);
+                }
             } else if (isRunning(component)) {
-                runningReferences.add(component);
+                activeReferences.add(component);
             }
         }
 
-        runningReferences.addAll(getRunningIndirectReferences(serviceNodes));
-        return runningReferences;
+        activeReferences.addAll(getActiveIndirectReferences(serviceNodes));
+        return activeReferences;
     }
 
-    private Set<ConfiguredComponent> getRunningIndirectReferences(final Set<ControllerServiceNode> referencingServices) {
+    private Set<ConfiguredComponent> getActiveIndirectReferences(final Set<ControllerServiceNode> referencingServices) {
         if (referencingServices.isEmpty()) {
             return Collections.emptySet();
         }
@@ -92,7 +97,7 @@ public class StandardControllerServiceReference implements ControllerServiceRefe
                 }
             }
 
-            references.addAll(getRunningIndirectReferences(serviceNodes));
+            references.addAll(getActiveIndirectReferences(serviceNodes));
         }
 
         return references;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
index 9b70581..0c472c8 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ReportingTaskWrapper.java
@@ -19,15 +19,13 @@ package org.apache.nifi.controller.tasks;
 import org.apache.nifi.annotation.lifecycle.OnStopped;
 import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.scheduling.ScheduleState;
+import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.processor.SimpleProcessLogger;
 import org.apache.nifi.util.ReflectionUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class ReportingTaskWrapper implements Runnable {
 
-    private static final Logger logger = LoggerFactory.getLogger(ReportingTaskWrapper.class);
-
     private final ReportingTaskNode taskNode;
     private final ScheduleState scheduleState;
 
@@ -43,20 +41,23 @@ public class ReportingTaskWrapper implements Runnable {
         try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
             taskNode.getReportingTask().onTrigger(taskNode.getReportingContext());
         } catch (final Throwable t) {
-            logger.error("Error running task {} due to {}", taskNode.getReportingTask(), t.toString());
-            if (logger.isDebugEnabled()) {
-                logger.error("", t);
+            final ComponentLog componentLog = new SimpleProcessLogger(taskNode.getIdentifier(), taskNode.getReportingTask());
+            componentLog.error("Error running task {} due to {}", new Object[] {taskNode.getReportingTask(), t.toString()});
+            if (componentLog.isDebugEnabled()) {
+                componentLog.error("", t);
             }
         } finally {
-            // if the processor is no longer scheduled to run and this is the last thread,
-            // invoke the OnStopped methods
-            if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) {
-                try (final NarCloseable x = NarCloseable.withNarLoader()) {
-                    ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, taskNode.getReportingTask(), taskNode.getReportingContext());
+            try {
+                // if the reporting task is no longer scheduled to run and this is the last thread,
+                // invoke the OnStopped methods
+                if (!scheduleState.isScheduled() && scheduleState.getActiveThreadCount() == 1 && scheduleState.mustCallOnStoppedMethods()) {
+                    try (final NarCloseable x = NarCloseable.withNarLoader()) {
+                        ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnStopped.class, org.apache.nifi.processor.annotation.OnStopped.class, taskNode.getReportingTask(), taskNode.getConfigurationContext());
+                    }
                 }
+            } finally {
+                scheduleState.decrementActiveThreadCount();
             }
-
-            scheduleState.decrementActiveThreadCount();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/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..9a8ad28 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,66 @@ 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.getComments());
+    	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.getComments());
+    	builder.append(dto.getSchedulingPeriod());
+    	builder.append(dto.getSchedulingStrategy());
+    	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/e9647717/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 3cd5853..216d015 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;
 
@@ -329,7 +331,8 @@ public final class StandardProcessGroup implements ProcessGroup {
     private void shutdown(final ProcessGroup procGroup) {
         for (final ProcessorNode node : procGroup.getProcessors()) {
             try (final NarCloseable x = NarCloseable.withNarLoader()) {
-                ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor());
+                final StandardProcessContext processContext = new StandardProcessContext(node, controllerServiceProvider, encryptor);
+                ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnShutdown.class, org.apache.nifi.processor.annotation.OnShutdown.class, node.getProcessor(), processContext);
             }
         }
 
@@ -674,6 +677,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();
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
index 4f3afaf..8957314 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
@@ -19,15 +19,12 @@ package org.apache.nifi.persistence;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.util.List;
 
 import org.apache.nifi.cluster.protocol.DataFlow;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.FlowSerializationException;
 import org.apache.nifi.controller.FlowSynchronizationException;
-import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.UninheritableFlowException;
-import org.apache.nifi.controller.service.ControllerServiceNode;
 
 /**
  * Interface to define service methods for FlowController configuration.
@@ -110,26 +107,4 @@ public interface FlowConfigurationDAO {
      */
     void save(FlowController flow, boolean archive) throws IOException;
 
-    /**
-     * Instantiates and schedules all controller tasks from the file used in the
-     * constructor
-     *
-     * @param controller
-     * @return 
-     * @throws java.io.IOException
-     * @returns all of the ReportingTasks that were instantiated & scheduled
-     */
-    List<ReportingTaskNode> loadReportingTasks(FlowController controller) throws IOException;
-
-    /**
-     * Instantiates all controller services from the file used in the
-     * constructor
-     *
-     * @param controller
-     * @return 
-     * @throws java.io.IOException
-     * @returns all of the ReportingTasks that were instantiated & scheduled
-     */
-    List<ControllerServiceNode> loadControllerServices(FlowController controller) throws IOException;
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
index c11aa72..b93ae8a 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/StandardXMLFlowConfigurationDAO.java
@@ -21,72 +21,36 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.net.URL;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
 
-import javax.xml.XMLConstants;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.transform.dom.DOMSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import javax.xml.validation.Validator;
-
 import org.apache.nifi.cluster.protocol.DataFlow;
-import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.FlowSerializationException;
 import org.apache.nifi.controller.FlowSynchronizationException;
 import org.apache.nifi.controller.FlowSynchronizer;
-import org.apache.nifi.controller.ReportingTaskNode;
 import org.apache.nifi.controller.StandardFlowSerializer;
 import org.apache.nifi.controller.StandardFlowSynchronizer;
 import org.apache.nifi.controller.UninheritableFlowException;
-import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException;
-import org.apache.nifi.controller.reporting.StandardReportingInitializationContext;
-import org.apache.nifi.controller.service.ControllerServiceLoader;
-import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.encrypt.StringEncryptor;
-import org.apache.nifi.util.file.FileUtils;
-import org.apache.nifi.nar.NarCloseable;
-import org.apache.nifi.reporting.InitializationException;
-import org.apache.nifi.reporting.ReportingInitializationContext;
-import org.apache.nifi.reporting.ReportingTask;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.util.DomUtils;
 import org.apache.nifi.util.NiFiProperties;
-
+import org.apache.nifi.util.file.FileUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.w3c.dom.DOMException;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-import org.xml.sax.SAXParseException;
 
 public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationDAO {
 
     public static final String CONFIGURATION_ARCHIVE_DIR_KEY = "nifi.flow.configuration.archive.dir";
 
     private final Path flowXmlPath;
-    private final Path taskConfigXmlPath;
-    private final ControllerServiceLoader servicerLoader;
     private final StringEncryptor encryptor;
 
     private static final Logger LOG = LoggerFactory.getLogger(StandardXMLFlowConfigurationDAO.class);
 
-    public StandardXMLFlowConfigurationDAO(final Path flowXml, final Path taskConfigXml, final Path serviceConfigXml, final StringEncryptor encryptor) throws IOException {
+    public StandardXMLFlowConfigurationDAO(final Path flowXml, final StringEncryptor encryptor) throws IOException {
         final File flowXmlFile = flowXml.toFile();
         if (!flowXmlFile.exists()) {
             Files.createDirectories(flowXml.getParent());
@@ -96,14 +60,7 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
             throw new IOException(flowXml + " exists but you have insufficient read/write privileges");
         }
 
-        final File taskConfigXmlFile = Objects.requireNonNull(taskConfigXml).toFile();
-        if ((!taskConfigXmlFile.exists() || !taskConfigXmlFile.canRead())) {
-            throw new IOException(taskConfigXml + " does not appear to exist or cannot be read. Cannot load configuration.");
-        }
-
         this.flowXmlPath = flowXml;
-        this.taskConfigXmlPath = taskConfigXml;
-        this.servicerLoader = new ControllerServiceLoader(serviceConfigXml);
         this.encryptor = encryptor;
     }
 
@@ -198,148 +155,4 @@ public final class StandardXMLFlowConfigurationDAO implements FlowConfigurationD
         }
     }
 
-    @Override
-    public List<ReportingTaskNode> loadReportingTasks(final FlowController controller) {
-        final List<ReportingTaskNode> tasks = new ArrayList<>();
-        if (taskConfigXmlPath == null) {
-            LOG.info("No reporting tasks to start");
-            return tasks;
-        }
-
-        try {
-            final URL schemaUrl = getClass().getResource("/ReportingTaskConfiguration.xsd");
-            final Document document = parse(taskConfigXmlPath.toFile(), schemaUrl);
-
-            final NodeList tasksNodes = document.getElementsByTagName("tasks");
-            final Element tasksElement = (Element) tasksNodes.item(0);
-
-            for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "task")) {
-                //add global properties common to all tasks
-                Map<String, String> properties = new HashMap<>();
-
-                //get properties for the specific reporting task - id, name, class,
-                //and schedulingPeriod must be set
-                final String taskId = DomUtils.getChild(taskElement, "id").getTextContent().trim();
-                final String taskName = DomUtils.getChild(taskElement, "name").getTextContent().trim();
-
-                final List<Element> schedulingStrategyNodeList = DomUtils.getChildElementsByTagName(taskElement, "schedulingStrategy");
-                String schedulingStrategyValue = SchedulingStrategy.TIMER_DRIVEN.name();
-                if (schedulingStrategyNodeList.size() == 1) {
-                    final String specifiedValue = schedulingStrategyNodeList.get(0).getTextContent();
-
-                    try {
-                        schedulingStrategyValue = SchedulingStrategy.valueOf(specifiedValue).name();
-                    } catch (final Exception e) {
-                        throw new RuntimeException("Cannot start Reporting Task with id " + taskId + " because its Scheduling Strategy does not have a valid value", e);
-                    }
-                }
-
-                final SchedulingStrategy schedulingStrategy = SchedulingStrategy.valueOf(schedulingStrategyValue);
-                final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim();
-                final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim();
-
-                //optional task-specific properties
-                for (final Element optionalProperty : DomUtils.getChildElementsByTagName(taskElement, "property")) {
-                    final String name = optionalProperty.getAttribute("name");
-                    final String value = optionalProperty.getTextContent().trim();
-                    properties.put(name, value);
-                }
-
-                //set the class to be used for the configured reporting task
-                final ReportingTaskNode reportingTaskNode;
-                try {
-                    reportingTaskNode = controller.createReportingTask(taskClass, taskId);
-                } catch (final ReportingTaskInstantiationException e) {
-                    LOG.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e});
-                    if (LOG.isDebugEnabled()) {
-                        LOG.error("", e);
-                    }
-                    continue;
-                }
-
-                reportingTaskNode.setName(taskName);
-                reportingTaskNode.setScheduldingPeriod(taskSchedulingPeriod);
-                reportingTaskNode.setSchedulingStrategy(schedulingStrategy);
-
-                final ReportingTask reportingTask = reportingTaskNode.getReportingTask();
-
-                final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, schedulingStrategy, taskSchedulingPeriod, controller);
-                reportingTask.initialize(config);
-
-                final Map<PropertyDescriptor, String> resolvedProps;
-                try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) {
-                    resolvedProps = new HashMap<>();
-                    for (final Map.Entry<String, String> entry : properties.entrySet()) {
-                        final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey());
-                        resolvedProps.put(descriptor, entry.getValue());
-                    }
-                }
-
-                for (final Map.Entry<PropertyDescriptor, String> entry : resolvedProps.entrySet()) {
-                    reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue());
-                }
-
-                tasks.add(reportingTaskNode);
-                controller.startReportingTask(reportingTaskNode);
-            }
-        } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) {
-            LOG.error("Unable to load reporting tasks from {} due to {}", new Object[]{taskConfigXmlPath, t});
-            if (LOG.isDebugEnabled()) {
-                LOG.error("", t);
-            }
-        }
-
-        return tasks;
-    }
-
-    @Override
-    public List<ControllerServiceNode> loadControllerServices(final FlowController controller) throws IOException {
-        return servicerLoader.loadControllerServices(controller);
-    }
-
-    private Document parse(final File xmlFile, final URL schemaUrl) throws SAXException, ParserConfigurationException, IOException {
-        final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-        final Schema schema = schemaFactory.newSchema(schemaUrl);
-        final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
-        docFactory.setSchema(schema);
-        final DocumentBuilder builder = docFactory.newDocumentBuilder();
-
-        builder.setErrorHandler(new org.xml.sax.ErrorHandler() {
-            @Override
-            public void fatalError(final SAXParseException err) throws SAXException {
-                LOG.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage());
-                if (LOG.isDebugEnabled()) {
-                    LOG.error("Error Stack Dump", err);
-                }
-                throw err;
-            }
-
-            @Override
-            public void error(final SAXParseException err) throws SAXParseException {
-                LOG.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage());
-                if (LOG.isDebugEnabled()) {
-                    LOG.error("Error Stack Dump", err);
-                }
-                throw err;
-            }
-
-            @Override
-            public void warning(final SAXParseException err) throws SAXParseException {
-                LOG.warn(" Config file line " + err.getLineNumber() + ", uri " + err.getSystemId() + " : message : " + err.getMessage());
-                if (LOG.isDebugEnabled()) {
-                    LOG.warn("Warning stack dump", err);
-                }
-                throw err;
-            }
-        });
-
-        // build the docuemnt
-        final Document document = builder.parse(xmlFile);
-
-        // ensure schema compliance
-        final Validator validator = schema.newValidator();
-        validator.validate(new DOMSource(document));
-
-        return document;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
index 88f1790..25d8f10 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/SimpleProcessLogger.java
@@ -28,16 +28,16 @@ public class SimpleProcessLogger implements ProcessorLog {
 
     private final Logger logger;
     private final LogRepository logRepository;
-    private final Processor processor;
+    private final Object component;
 
-    public SimpleProcessLogger(final String processorId, final Processor processor) {
-        this.logger = LoggerFactory.getLogger(processor.getClass());
-        this.logRepository = LogRepositoryFactory.getRepository(processorId);
-        this.processor = processor;
+    public SimpleProcessLogger(final String componentId, final Object component) {
+        this.logger = LoggerFactory.getLogger(component.getClass());
+        this.logRepository = LogRepositoryFactory.getRepository(componentId);
+        this.component = component;
     }
 
     private Object[] addProcessor(final Object[] originalArgs) {
-        return prependToArgs(originalArgs, processor);
+        return prependToArgs(originalArgs, component);
     }
 
     private Object[] prependToArgs(final Object[] originalArgs, final Object... toAdd) {
@@ -63,7 +63,7 @@ public class SimpleProcessLogger implements ProcessorLog {
 
     @Override
     public void warn(final String msg, final Throwable t) {
-        warn("{} " + msg, new Object[]{processor}, t);
+        warn("{} " + msg, new Object[]{component}, t);
     }
 
     @Override
@@ -93,15 +93,15 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void warn(String msg) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
-        logger.warn(msg, processor);
+        final Object[] os = {component};
+        logger.warn(msg, component);
         logRepository.addLogMessage(LogLevel.WARN, msg, os);
     }
 
     @Override
     public void trace(String msg, Throwable t) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
         logger.trace(msg, os, t);
         logRepository.addLogMessage(LogLevel.TRACE, msg, os, t);
     }
@@ -117,7 +117,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void trace(String msg) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
         logger.trace(msg, os);
         logRepository.addLogMessage(LogLevel.TRACE, msg, os);
     }
@@ -160,7 +160,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void info(String msg, Throwable t) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
 
         logger.info(msg, os);
         if (logger.isDebugEnabled()) {
@@ -181,7 +181,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void info(String msg) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
 
         logger.info(msg, os);
         logRepository.addLogMessage(LogLevel.INFO, msg, os);
@@ -207,7 +207,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void error(String msg, Throwable t) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
 
         logger.error(msg, os, t);
         if (logger.isDebugEnabled()) {
@@ -231,7 +231,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void error(String msg) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
 
         logger.error(msg, os);
         logRepository.addLogMessage(LogLevel.ERROR, msg, os);
@@ -239,7 +239,7 @@ public class SimpleProcessLogger implements ProcessorLog {
 
     private Object[] addProcessorAndThrowable(final Object[] os, final Throwable t) {
         final Object[] modifiedArgs = new Object[os.length + 2];
-        modifiedArgs[0] = processor.toString();
+        modifiedArgs[0] = component.toString();
         for (int i = 0; i < os.length; i++) {
             modifiedArgs[i + 1] = os[i];
         }
@@ -263,7 +263,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void debug(String msg, Throwable t) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
 
         logger.debug(msg, os, t);
         logRepository.addLogMessage(LogLevel.DEBUG, msg, os, t);
@@ -298,7 +298,7 @@ public class SimpleProcessLogger implements ProcessorLog {
     @Override
     public void debug(String msg) {
         msg = "{} " + msg;
-        final Object[] os = {processor};
+        final Object[] os = {component};
 
         logger.debug(msg, os);
         logRepository.addLogMessage(LogLevel.DEBUG, msg, os);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index cd0d31c..d14a459 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@ -142,6 +142,11 @@ public class StandardProcessContext implements ProcessContext, ControllerService
     }
 
     @Override
+    public boolean isControllerServiceEnabling(final String serviceIdentifier) {
+        return controllerServiceProvider.isControllerServiceEnabling(serviceIdentifier);
+    }
+    
+    @Override
     public ControllerServiceLookup getControllerServiceLookup() {
         return this;
     }
@@ -170,4 +175,9 @@ public class StandardProcessContext implements ProcessContext, ControllerService
         return set;
     }
     
+    @Override
+    public String getControllerServiceName(final String serviceIdentifier) {
+    	return controllerServiceProvider.getControllerServiceName(serviceIdentifier);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
index ac58504..c37a80d 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardSchedulingContext.java
@@ -25,6 +25,7 @@ import org.apache.nifi.controller.ControllerServiceLookup;
 import org.apache.nifi.controller.ProcessorNode;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
+import org.apache.nifi.controller.service.ControllerServiceState;
 
 public class StandardSchedulingContext implements SchedulingContext {
 
@@ -45,8 +46,8 @@ public class StandardSchedulingContext implements SchedulingContext {
             throw new IllegalArgumentException("Cannot lease Controller Service because no Controller Service exists with identifier " + identifier);
         }
 
-        if (serviceNode.isDisabled()) {
-            throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getProxiedControllerService() + " is currently disabled");
+        if ( serviceNode.getState() != ControllerServiceState.ENABLED ) {
+            throw new IllegalStateException("Cannot lease Controller Service because Controller Service " + serviceNode.getProxiedControllerService() + " is not currently enabled");
         }
 
         if (!serviceNode.isValid()) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
index 99322be..c51cb9a 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContext.java
@@ -20,6 +20,7 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.nifi.attribute.expression.language.PreparedQuery;
 import org.apache.nifi.attribute.expression.language.Query;
@@ -41,11 +42,17 @@ public class StandardValidationContext implements ValidationContext {
     private final Map<PropertyDescriptor, PreparedQuery> preparedQueries;
     private final Map<String, Boolean> expressionLanguageSupported;
     private final String annotationData;
+    private final Set<String> serviceIdentifiersToNotValidate;
 
     public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Map<PropertyDescriptor, String> properties, final String annotationData) {
+        this(controllerServiceProvider, Collections.<String>emptySet(), properties, annotationData);
+    }
+    
+    public StandardValidationContext(final ControllerServiceProvider controllerServiceProvider, final Set<String> serviceIdentifiersToNotValidate, final Map<PropertyDescriptor, String> properties, final String annotationData) {
         this.controllerServiceProvider = controllerServiceProvider;
         this.properties = new HashMap<>(properties);
         this.annotationData = annotationData;
+        this.serviceIdentifiersToNotValidate = serviceIdentifiersToNotValidate;
 
         preparedQueries = new HashMap<>(properties.size());
         for (final Map.Entry<PropertyDescriptor, String> entry : properties.entrySet()) {
@@ -101,6 +108,11 @@ public class StandardValidationContext implements ValidationContext {
     public ControllerServiceLookup getControllerServiceLookup() {
         return controllerServiceProvider;
     }
+
+    @Override
+    public boolean isValidationRequired(final ControllerService service) {
+        return !serviceIdentifiersToNotValidate.contains(service.getIdentifier());
+    }
     
     @Override
     public boolean isExpressionLanguagePresent(final String value) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java
index e172f93..c3df987 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardValidationContextFactory.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.processor;
 
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.components.ValidationContext;
@@ -36,4 +37,8 @@ public class StandardValidationContextFactory implements ValidationContextFactor
         return new StandardValidationContext(serviceProvider, properties, annotationData);
     }
 
+    @Override
+    public ValidationContext newValidationContext(final Set<String> serviceIdentifiersToNotValidate, final Map<PropertyDescriptor, String> properties, final String annotationData) {
+        return new StandardValidationContext(serviceProvider, serviceIdentifiersToNotValidate, properties, annotationData);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java
index c8b6573..da4f04d 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/DomUtils.java
@@ -25,6 +25,16 @@ import org.w3c.dom.NodeList;
 
 public class DomUtils {
 
+    public static String getChildText(final Element element, final String tagName) {
+        final Element childElement = getChild(element, tagName);
+        if ( childElement == null ) {
+            return null;
+        }
+        
+        final String text = childElement.getTextContent();
+        return (text == null) ? null : text.trim();
+    }
+    
     public static Element getChild(final Element element, final String tagName) {
         final List<Element> children = getChildElementsByTagName(element, tagName);
         if (children.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd
deleted file mode 100644
index d3efed1..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ControllerServiceConfiguration.xsd
+++ /dev/null
@@ -1,61 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  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.
--->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" version="1.0">
-    <xs:element name="services" type="ControllerServices"/>
-
-    <xs:complexType name="ControllerServices">
-        <xs:sequence>
-
-            <!-- Each "processor" defines the actual dataflow work horses that make dataflow happen-->
-            <xs:element name="service" type="ControllerServiceType" minOccurs="0" maxOccurs="unbounded"/>
-
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- the Controller Task "id" is a key that should be valid within each flowController-->
-    <xs:complexType name="ControllerServiceType">
-        <xs:sequence>
-
-            <!-- The "id" is a name used to uniquely identify the Controller Task. -->
-            <xs:element name="identifier" type="NonEmptyStringType"/>
-
-            <xs:element name="name" type="xs:string" minOccurs="0" maxOccurs="1" />
-
-            <!-- "class" is the actual Java class that performs the type of controller task desired-->
-            <xs:element name="class" type="NonEmptyStringType"/>
-
-            <!-- "optionalTaskProperty" are properties that may exist and offer further information about a task.
-            For instance, information about where a specific task should send information. -->
-            <xs:element name="property" type="PropertyType" minOccurs="0" maxOccurs="unbounded" />
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- Name/Value properties-->
-    <xs:complexType name="PropertyType">
-        <xs:simpleContent>
-            <xs:extension base="xs:string">
-                <xs:attribute name="name" type="xs:string"></xs:attribute>
-            </xs:extension>
-        </xs:simpleContent>
-    </xs:complexType>
-
-
-    <xs:simpleType name="NonEmptyStringType">
-        <xs:restriction base="xs:string">
-            <xs:minLength value="1"/>
-        </xs:restriction>
-    </xs:simpleType>
-</xs:schema>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
index 1e6c25c..00c71ac 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/FlowConfiguration.xsd
@@ -28,6 +28,10 @@
 
             <!-- Groupings of Processors/Ports -->
             <xs:element name="rootGroup" type="RootProcessGroupType" />
+            
+            <xs:element name="controllerServices" type="ControllerServicesType" minOccurs="0" maxOccurs="1" />
+            
+            <xs:element name="reportingTasks" type="ReportingTasksType" minOccurs="0" maxOccurs="1" />
         </xs:sequence>
     </xs:complexType>
 	
@@ -58,11 +62,6 @@
             IFF schedulingStrategy is EVENT_DRIVEN -->
             <xs:element name="maxConcurrentTasks" type="xs:nonNegativeInteger"/>
 
-            <!-- "schedulingPeriodSeconds" is the maximum number of seconds that should elapse
-            between successive executions of each task for this configured processor.  If a
-            task takes longer than the period specified then the next execution of this
-            task will take place immediately after termination of the previous run and as soon
-            as their is an available thread.-->
             <xs:element name="schedulingPeriod" type="NonEmptyStringType"/>
             
             <xs:element name="penalizationPeriod" type="TimePeriod" />
@@ -332,4 +331,44 @@
             <xs:enumeration value="CRON_DRIVEN"></xs:enumeration>
         </xs:restriction>
     </xs:simpleType>
+    
+    <xs:complexType name="ControllerServicesType">
+    	<xs:sequence>
+    		<xs:element name="controllerService" type="ControllerServiceType" minOccurs="0" maxOccurs="unbounded" />
+    	</xs:sequence>
+    </xs:complexType>
+    
+    <xs:complexType name="ControllerServiceType">
+    	<xs:sequence>
+    		<xs:element name="id" type="NonEmptyStringType" />
+    		<xs:element name="name" type="NonEmptyStringType" />
+    		<xs:element name="comment" type="xs:string" />
+    		<xs:element name="class" type="NonEmptyStringType" />
+    		<xs:element name="enabled" type="xs:boolean" />
+    		
+            <xs:element name="property" type="PropertyType" minOccurs="0" maxOccurs="unbounded"/>
+            <xs:element name="annotationData" type="xs:string" minOccurs="0" maxOccurs="1" />
+    	</xs:sequence>
+    </xs:complexType>
+    
+    <xs:complexType name="ReportingTasksType">
+    	<xs:sequence>
+    		<xs:element name="reportingTask" type="ReportingTaskType" minOccurs="0" maxOccurs="unbounded" />
+    	</xs:sequence>
+    </xs:complexType>
+    
+    <xs:complexType name="ReportingTaskType">
+    	<xs:sequence>
+    		<xs:element name="id" type="NonEmptyStringType" />
+    		<xs:element name="name" type="NonEmptyStringType" />
+    		<xs:element name="comment" type="xs:string" />
+    		<xs:element name="class" type="NonEmptyStringType" />
+            <xs:element name="schedulingPeriod" type="NonEmptyStringType"/>
+            <xs:element name="schedulingState" type="ScheduledState" />
+            <xs:element name="schedulingStrategy" type="SchedulingStrategy" />
+    		
+            <xs:element name="property" type="PropertyType" minOccurs="0" maxOccurs="unbounded"/>
+            <xs:element name="annotationData" type="xs:string" minOccurs="0" maxOccurs="1" />
+    	</xs:sequence>
+    </xs:complexType>
 </xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd
deleted file mode 100644
index dcf1090..0000000
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/resources/ReportingTaskConfiguration.xsd
+++ /dev/null
@@ -1,87 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  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.
--->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema" version="1.0">
-    <xs:element name="tasks" type="ReportingTasks"/>
-
-    <xs:complexType name="ReportingTasks">
-        <xs:sequence>
-
-            <!-- properties that may exist and offer further information about all tasks.
-            For instance, possibly the system that is sending the information. -->
-            <xs:element name="property" type="PropertyType" minOccurs="0" maxOccurs="unbounded"/>
-
-            <!-- Each "processor" defines the actual dataflow work horses that make dataflow happen-->
-            <xs:element name="task" type="ReportingTaskType" minOccurs="0" maxOccurs="unbounded"/>
-
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- the Controller Task "id" is a key that should be valid within each flowController-->
-    <xs:complexType name="ReportingTaskType">
-        <xs:sequence>
-
-            <!-- The "id" is a name used to uniquely identify the Controller Task. -->
-            <xs:element name="id" type="NonEmptyStringType"/>
-
-            <!-- The "name" is a nicely displayable description of the Controller Task's duty-->
-            <xs:element name="name" type="NonEmptyStringType"/>
-
-            <!-- "class" is the actual Java class that performs the type of controller task desired-->
-            <xs:element name="class" type="NonEmptyStringType"/>
-
-            <!-- 
-                "schedulingPeriod" is the amount of time that should elapse between successive 
-                executions of this task. The timer starts as soon as an execution finishes
-            -->
-            <xs:element name="schedulingPeriod" type="NonEmptyStringType"/>
-            
-            <xs:element name="schedulingStrategy" type="SchedulingStrategyType" minOccurs="0" default="TIMER_DRIVEN" />
-
-            <!-- "optionalTaskProperty" are properties that may exist and offer further information about a task.
-            For instance, information about where a specific task should send information. -->
-            <xs:element name="property" type="PropertyType" minOccurs="0" maxOccurs="unbounded" />
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- Name/Value properties-->
-    <xs:complexType name="PropertyType">
-        <xs:simpleContent>
-            <xs:extension base="xs:string">
-                <xs:attribute name="name" type="xs:string"></xs:attribute>
-            </xs:extension>
-        </xs:simpleContent>
-    </xs:complexType>
-
-    <xs:simpleType name="SchedulingStrategyType">
-        <xs:restriction base="xs:string">
-            <xs:enumeration value="TIMER_DRIVEN"></xs:enumeration>
-            <xs:enumeration value="CRON_DRIVEN"></xs:enumeration>
-        </xs:restriction>
-    </xs:simpleType>
-
-    <xs:simpleType name="NonEmptyStringType">
-        <xs:restriction base="xs:string">
-            <xs:minLength value="1"/>
-        </xs:restriction>
-    </xs:simpleType>
-    
-    <xs:simpleType name="TimePeriod">
-        <xs:restriction base="xs:string">
-            <xs:pattern value="\d+\s*(ns|nano|nanos|nanoseconds|ms|milli|millis|milliseconds|s|sec|secs|seconds|m|min|mins|minutes|h|hr|hrs|hours|d|day|days)"></xs:pattern>
-        </xs:restriction>
-    </xs:simpleType>
-    
-</xs:schema>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
index b889bc8..7fef706 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/StandardControllerServiceProviderTest.java
@@ -43,7 +43,7 @@ public class StandardControllerServiceProviderTest {
     public void setup() throws Exception {
     	String id = "id";
     	String clazz = "org.apache.nifi.controller.service.util.TestControllerService";  
-    	ControllerServiceProvider provider = new StandardControllerServiceProvider();
+    	ControllerServiceProvider provider = new StandardControllerServiceProvider(null, null);
     	ControllerServiceNode node = provider.createControllerService(clazz,id,true);
     	proxied = node.getProxiedControllerService();
     	implementation = node.getControllerServiceImplementation();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
new file mode 100644
index 0000000..3dc1752
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/TestStandardControllerServiceProvider.java
@@ -0,0 +1,385 @@
+/*
+ * 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.controller.service;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.nifi.controller.ProcessScheduler;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.StandardProcessorNode;
+import org.apache.nifi.controller.service.mock.DummyProcessor;
+import org.apache.nifi.controller.service.mock.ServiceA;
+import org.apache.nifi.controller.service.mock.ServiceB;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.processor.StandardProcessorInitializationContext;
+import org.apache.nifi.processor.StandardValidationContextFactory;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestStandardControllerServiceProvider {
+
+    private ProcessScheduler createScheduler() {
+        final ProcessScheduler scheduler = Mockito.mock(ProcessScheduler.class);
+        Mockito.doAnswer(new Answer<Object>() {
+            @Override
+            public Object answer(final InvocationOnMock invocation) throws Throwable {
+                final ControllerServiceNode node = (ControllerServiceNode) invocation.getArguments()[0];
+                node.verifyCanEnable();
+                node.setState(ControllerServiceState.ENABLED);
+                return null;
+            }
+        }).when(scheduler).enableControllerService(Mockito.any(ControllerServiceNode.class));
+        
+        Mockito.doAnswer(new Answer<Object>() {
+            @Override
+            public Object answer(final InvocationOnMock invocation) throws Throwable {
+                final ControllerServiceNode node = (ControllerServiceNode) invocation.getArguments()[0];
+                node.verifyCanDisable();
+                node.setState(ControllerServiceState.DISABLED);
+                return null;
+            }
+        }).when(scheduler).disableControllerService(Mockito.any(ControllerServiceNode.class));
+        
+        return scheduler;
+    }
+    
+    @Test
+    public void testDisableControllerService() {
+        final ProcessScheduler scheduler = createScheduler();
+        final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null);
+        
+        final ControllerServiceNode serviceNode = provider.createControllerService(ServiceB.class.getName(), "B", false);
+        provider.enableControllerService(serviceNode);
+        provider.disableControllerService(serviceNode);
+    }
+    
+    @Test
+    public void testEnableDisableWithReference() {
+        final ProcessScheduler scheduler = createScheduler();
+        final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null);
+        
+        final ControllerServiceNode serviceNodeB = provider.createControllerService(ServiceB.class.getName(), "B", false);
+        final ControllerServiceNode serviceNodeA = provider.createControllerService(ServiceA.class.getName(), "A", false);
+        
+        serviceNodeA.setProperty(ServiceA.OTHER_SERVICE.getName(), "B");
+        
+        try {
+            provider.enableControllerService(serviceNodeA);
+            Assert.fail("Was able to enable Service A but Service B is disabled.");
+        } catch (final IllegalStateException expected) {
+        }
+        
+        provider.enableControllerService(serviceNodeB);
+        provider.enableControllerService(serviceNodeA);
+        
+        try {
+            provider.disableControllerService(serviceNodeB);
+            Assert.fail("Was able to disable Service B but Service A is enabled and references B");
+        } catch (final IllegalStateException expected) {
+        }
+        
+        provider.disableControllerService(serviceNodeA);
+        provider.disableControllerService(serviceNodeB);
+    }
+    
+    
+    @Test
+    public void testEnableReferencingServicesGraph() {
+        final ProcessScheduler scheduler = createScheduler();
+        final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null);
+        
+        // build a graph of controller services with dependencies as such:
+        //
+        // A -> B -> D
+        // C ---^----^
+        //
+        // In other words, A references B, which references D.
+        // AND
+        // C references B and D.
+        //
+        // So we have to verify that if D is enabled, when we enable its referencing services,
+        // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so
+        // until B is first enabled so ensure that we enable B first.
+        
+        final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false);
+        final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", false);
+        final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false);
+        final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false);
+        
+        serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+        serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4");
+        serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+        serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4");
+        
+        provider.enableControllerService(serviceNode4);
+        provider.enableReferencingServices(serviceNode4);
+        
+        assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState());
+    }
+    
+    
+    @Test
+    public void testStartStopReferencingComponents() {
+        final ProcessScheduler scheduler = createScheduler();
+        final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(scheduler, null);
+        
+        // build a graph of reporting tasks and controller services with dependencies as such:
+        //
+        // Processor P1 -> A -> B -> D
+        // Processor P2 -> C ---^----^
+        //
+        // In other words, Processor P1 references Controller Service A, which references B, which references D.
+        // AND
+        // Processor P2 references Controller Service C, which references B and D.
+        //
+        // So we have to verify that if D is enabled, when we enable its referencing services,
+        // we enable C and B, even if we attempt to enable C before B... i.e., if we try to enable C, we cannot do so
+        // until B is first enabled so ensure that we enable B first.
+        
+        final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false);
+        final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceA.class.getName(), "2", false);
+        final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false);
+        final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false);
+        
+        final ProcessGroup mockProcessGroup = Mockito.mock(ProcessGroup.class);
+        Mockito.doAnswer(new Answer<Object>() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                final ProcessorNode procNode = (ProcessorNode) invocation.getArguments()[0];
+                procNode.verifyCanStart();
+                procNode.setScheduledState(ScheduledState.RUNNING);
+                return null;
+            }
+        }).when(mockProcessGroup).startProcessor(Mockito.any(ProcessorNode.class));
+        
+        Mockito.doAnswer(new Answer<Object>() {
+            @Override
+            public Object answer(final InvocationOnMock invocation) throws Throwable {
+                final ProcessorNode procNode = (ProcessorNode) invocation.getArguments()[0];
+                procNode.verifyCanStop();
+                procNode.setScheduledState(ScheduledState.STOPPED);
+                return null;
+            }
+        }).when(mockProcessGroup).stopProcessor(Mockito.any(ProcessorNode.class));
+        
+        final String id1 = UUID.randomUUID().toString();
+        final ProcessorNode procNodeA = new StandardProcessorNode(new DummyProcessor(), id1,
+                new StandardValidationContextFactory(provider), scheduler, provider);
+        procNodeA.getProcessor().initialize(new StandardProcessorInitializationContext(id1, null, provider));
+        procNodeA.setProperty(DummyProcessor.SERVICE.getName(), "1");
+        procNodeA.setProcessGroup(mockProcessGroup);
+        
+        final String id2 = UUID.randomUUID().toString();
+        final ProcessorNode procNodeB = new StandardProcessorNode(new DummyProcessor(),id2,
+                new StandardValidationContextFactory(provider), scheduler, provider);
+        procNodeB.getProcessor().initialize(new StandardProcessorInitializationContext(id2, null, provider));
+        procNodeB.setProperty(DummyProcessor.SERVICE.getName(), "3");
+        procNodeB.setProcessGroup(mockProcessGroup);
+        
+        serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+        serviceNode2.setProperty(ServiceA.OTHER_SERVICE.getName(), "4");
+        serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+        serviceNode3.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "4");
+        
+        provider.enableControllerService(serviceNode4);
+        provider.enableReferencingServices(serviceNode4);
+        provider.scheduleReferencingComponents(serviceNode4);
+        
+        assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState());
+        assertTrue(procNodeA.isRunning());
+        assertTrue(procNodeB.isRunning());
+        
+        // stop processors and verify results.
+        provider.unscheduleReferencingComponents(serviceNode4);
+        assertFalse(procNodeA.isRunning());
+        assertFalse(procNodeB.isRunning());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode3.getState());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode2.getState());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode1.getState());
+        
+        provider.disableReferencingServices(serviceNode4);
+        assertEquals(ControllerServiceState.DISABLED, serviceNode3.getState());
+        assertEquals(ControllerServiceState.DISABLED, serviceNode2.getState());
+        assertEquals(ControllerServiceState.DISABLED, serviceNode1.getState());
+        assertEquals(ControllerServiceState.ENABLED, serviceNode4.getState());
+        
+        provider.disableControllerService(serviceNode4);
+        assertEquals(ControllerServiceState.DISABLED, serviceNode4.getState());
+    }
+    
+    
+    @Test
+    public void testOrderingOfServices() {
+        final StandardControllerServiceProvider provider = new StandardControllerServiceProvider(null, null);
+        final ControllerServiceNode serviceNode1 = provider.createControllerService(ServiceA.class.getName(), "1", false);
+        final ControllerServiceNode serviceNode2 = provider.createControllerService(ServiceB.class.getName(), "2", false);
+
+        serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+
+        final Map<String, ControllerServiceNode> nodeMap = new LinkedHashMap<>();
+        nodeMap.put("1", serviceNode1);
+        nodeMap.put("2", serviceNode2);
+        
+        List<List<ControllerServiceNode>> branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(2, branches.size());
+        List<ControllerServiceNode> ordered = branches.get(0);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        assertTrue(ordered.get(1) == serviceNode1);
+        assertEquals(1, branches.get(1).size());
+        assertTrue(branches.get(1).get(0) == serviceNode2);
+        
+        nodeMap.clear();
+        nodeMap.put("2", serviceNode2);
+        nodeMap.put("1", serviceNode1);
+        
+        branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(2, branches.size());
+        ordered = branches.get(1);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        assertTrue(ordered.get(1) == serviceNode1);
+        assertEquals(1, branches.get(0).size());
+        assertTrue(branches.get(0).get(0) == serviceNode2);
+        
+        // add circular dependency on self.
+        nodeMap.clear();
+        serviceNode1.setProperty(ServiceA.OTHER_SERVICE_2.getName(), "1");
+        nodeMap.put("1", serviceNode1);
+        nodeMap.put("2", serviceNode2);
+        
+        branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(2, branches.size());
+        ordered = branches.get(0);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        assertTrue(ordered.get(1) == serviceNode1);
+        
+        nodeMap.clear();
+        nodeMap.put("2", serviceNode2);
+        nodeMap.put("1", serviceNode1);
+        branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(2, branches.size());
+        ordered = branches.get(1);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        assertTrue(ordered.get(1) == serviceNode1);
+        
+        // add circular dependency once removed. In this case, we won't actually be able to enable these because of the
+        // circular dependency because they will never be valid because they will always depend on a disabled service.
+        // But we want to ensure that the method returns successfully without throwing a StackOverflowException or anything
+        // like that.
+        nodeMap.clear();
+        final ControllerServiceNode serviceNode3 = provider.createControllerService(ServiceA.class.getName(), "3", false);
+        serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "3");
+        serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "1");
+        nodeMap.put("1", serviceNode1);
+        nodeMap.put("3", serviceNode3);
+        branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(2, branches.size());
+        ordered = branches.get(0);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode3);
+        assertTrue(ordered.get(1) == serviceNode1);
+        
+        nodeMap.clear();
+        nodeMap.put("3", serviceNode3);
+        nodeMap.put("1", serviceNode1);
+        branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(2, branches.size());
+        ordered = branches.get(1);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode3);
+        assertTrue(ordered.get(1) == serviceNode1);
+        
+        
+        // Add multiple completely disparate branches.
+        nodeMap.clear();
+        serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+        final ControllerServiceNode serviceNode4 = provider.createControllerService(ServiceB.class.getName(), "4", false);
+        final ControllerServiceNode serviceNode5 = provider.createControllerService(ServiceB.class.getName(), "5", false);
+        serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "4");
+        nodeMap.put("1", serviceNode1);
+        nodeMap.put("2", serviceNode2);
+        nodeMap.put("3", serviceNode3);
+        nodeMap.put("4", serviceNode4);
+        nodeMap.put("5", serviceNode5);
+        
+        branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(5, branches.size());
+
+        ordered = branches.get(0);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        assertTrue(ordered.get(1) == serviceNode1);
+        
+        assertEquals(1, branches.get(1).size());
+        assertTrue(branches.get(1).get(0) == serviceNode2);
+        
+        ordered = branches.get(2);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode4);
+        assertTrue(ordered.get(1) == serviceNode3);
+        
+        assertEquals(1, branches.get(3).size());
+        assertTrue(branches.get(3).get(0) == serviceNode4);
+        
+        assertEquals(1, branches.get(4).size());
+        assertTrue(branches.get(4).get(0) == serviceNode5);
+        
+        // create 2 branches both dependent on the same service
+        nodeMap.clear();
+        serviceNode1.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+        serviceNode3.setProperty(ServiceA.OTHER_SERVICE.getName(), "2");
+        nodeMap.put("1", serviceNode1);
+        nodeMap.put("2", serviceNode2);
+        nodeMap.put("3", serviceNode3);
+        
+        branches = StandardControllerServiceProvider.determineEnablingOrder(nodeMap);
+        assertEquals(3, branches.size());
+        
+        ordered = branches.get(0);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        assertTrue(ordered.get(1) == serviceNode1);
+        
+        ordered = branches.get(1);
+        assertEquals(1, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        
+        ordered = branches.get(2);
+        assertEquals(2, ordered.size());
+        assertTrue(ordered.get(0) == serviceNode2);
+        assertTrue(ordered.get(1) == serviceNode3);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java
new file mode 100644
index 0000000..615e172
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/DummyProcessor.java
@@ -0,0 +1,49 @@
+/*
+ * 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.controller.service.mock;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ControllerService;
+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;
+
+public class DummyProcessor extends AbstractProcessor {
+
+    public static final PropertyDescriptor SERVICE = new PropertyDescriptor.Builder()
+        .name("Controller Service")
+        .identifiesControllerService(ControllerService.class)
+        .required(true)
+        .build();
+    
+    
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(SERVICE);
+        return descriptors;
+    }
+    
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e9647717/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java
new file mode 100644
index 0000000..4918468
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/service/mock/ServiceA.java
@@ -0,0 +1,49 @@
+/*
+ * 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.controller.service.mock;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ControllerService;
+
+public class ServiceA extends AbstractControllerService {
+
+    public static final PropertyDescriptor OTHER_SERVICE = new PropertyDescriptor.Builder()
+        .name("Other Service")
+        .identifiesControllerService(ControllerService.class)
+        .required(true)
+        .build();
+    
+    public static final PropertyDescriptor OTHER_SERVICE_2 = new PropertyDescriptor.Builder()
+        .name("Other Service 2")
+        .identifiesControllerService(ControllerService.class)
+        .required(false)
+        .build();
+
+    
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(OTHER_SERVICE);
+        descriptors.add(OTHER_SERVICE_2);
+        return descriptors;
+    }
+    
+}