You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2015/02/19 22:31:43 UTC

[2/2] incubator-nifi git commit: NIFI-250: - Using controller service state field instead of enable flag. - Continuing to setup Reporting Tasks.

NIFI-250:
- Using controller service state field instead of enable flag.
- Continuing to setup Reporting Tasks.

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

Branch: refs/heads/NIFI-250
Commit: 573a8aa353bb47cbfc782d06786f31a93faa3d7a
Parents: a227fe4
Author: Matt Gilman <ma...@gmail.com>
Authored: Thu Feb 19 16:30:50 2015 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Feb 19 16:30:50 2015 -0500

----------------------------------------------------------------------
 .../nifi/admin/dao/impl/StandardActionDAO.java  |   4 +-
 .../java/org/apache/nifi/action/Component.java  |   3 +-
 .../nifi/audit/ControllerServiceAuditor.java    |  27 +-
 .../apache/nifi/audit/ReportingTaskAuditor.java | 353 +++++++++++++++++++
 .../org/apache/nifi/web/NiFiServiceFacade.java  |   8 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |   4 +-
 .../nifi/web/api/ControllerServiceResource.java |  53 ++-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |   7 +-
 .../nifi/web/dao/ControllerServiceDAO.java      |   8 +-
 .../dao/impl/StandardControllerServiceDAO.java  |  72 ++--
 .../web/dao/impl/StandardReportingTaskDAO.java  |   3 +-
 .../src/main/resources/nifi-web-api-context.xml |  16 +-
 .../nifi-framework/nifi-web/nifi-web-ui/pom.xml |   1 +
 .../src/main/webapp/WEB-INF/pages/canvas.jsp    |   1 +
 .../canvas/new-reporting-task-dialog.jsp        |  53 +++
 .../nifi-web-ui/src/main/webapp/css/canvas.css  |   1 +
 .../webapp/css/new-reporting-task-dialog.css    | 152 ++++++++
 .../js/nf/canvas/nf-controller-service.js       |  28 +-
 .../src/main/webapp/js/nf/canvas/nf-settings.js | 332 ++++++++++++++++-
 .../webapp/js/nf/history/nf-history-table.js    |   2 +-
 20 files changed, 1012 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
index 7114d42..34623f4 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java
@@ -601,7 +601,7 @@ public class StandardActionDAO implements ActionDAO {
 
                 // get the component details if appropriate
                 ComponentDetails componentDetails = null;
-                if (Component.Processor.equals(component) || Component.ControllerService.equals(component)) {
+                if (Component.Processor.equals(component) || Component.ControllerService.equals(component)  || Component.ReportingTask.equals(component)) {
                     componentDetails = getExtensionDetails(actionId);
                 } else if (Component.RemoteProcessGroup.equals(component)) {
                     componentDetails = getRemoteProcessGroupDetails(actionId);
@@ -675,7 +675,7 @@ public class StandardActionDAO implements ActionDAO {
 
                 // get the component details if appropriate
                 ComponentDetails componentDetails = null;
-                if (Component.Processor.equals(component) || Component.ControllerService.equals(component)) {
+                if (Component.Processor.equals(component) || Component.ControllerService.equals(component) || Component.ReportingTask.equals(component)) {
                     componentDetails = getExtensionDetails(actionId);
                 } else if (Component.RemoteProcessGroup.equals(component)) {
                     componentDetails = getRemoteProcessGroupDetails(actionId);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java
index aff724f..7f62c92 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-user-actions/src/main/java/org/apache/nifi/action/Component.java
@@ -28,5 +28,6 @@ public enum Component {
     RemoteProcessGroup,
 	Funnel,
     Connection,
-    ControllerService;
+    ControllerService,
+    ReportingTask;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
index 5c9e93e..f6cc131 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ControllerServiceAuditor.java
@@ -37,6 +37,7 @@ import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceReference;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.dao.ControllerServiceDAO;
 import org.aspectj.lang.ProceedingJoinPoint;
@@ -55,7 +56,6 @@ public class ControllerServiceAuditor extends NiFiAuditor {
 
     private static final String COMMENTS = "Comments";
     private static final String NAME = "Name";
-    private static final String AVAILABILITY = "Availability";
     private static final String ANNOTATION_DATA = "Annotation Data";
 
     /**
@@ -106,13 +106,12 @@ public class ControllerServiceAuditor extends NiFiAuditor {
         // determine the initial values for each property/setting thats changing
         ControllerServiceNode controllerService = controllerServiceDAO.getControllerService(controllerServiceDTO.getId());
         final Map<String, String> values = extractConfiguredPropertyValues(controllerService, controllerServiceDTO);
-        final boolean isDisabled = controllerService.isDisabled();
+        final boolean isDisabled = isDisabled(controllerService);
 
         // update the controller service state
         final ControllerServiceNode updatedControllerService = (ControllerServiceNode) proceedingJoinPoint.proceed();
 
         // if no exceptions were thrown, add the controller service action...
-        // get the updated verbose state
         controllerService = controllerServiceDAO.getControllerService(updatedControllerService.getIdentifier());
 
         // get the current user
@@ -183,7 +182,7 @@ public class ControllerServiceAuditor extends NiFiAuditor {
             }
 
             // determine the new executing state
-            final boolean updateIsDisabled = controllerService.isDisabled();
+            final boolean updateIsDisabled = isDisabled(updatedControllerService);
 
             // determine if the running state has changed and its not disabled
             if (isDisabled != updateIsDisabled) {
@@ -224,7 +223,7 @@ public class ControllerServiceAuditor extends NiFiAuditor {
      * @throws Throwable
      */
     @Around("within(org.apache.nifi.web.dao.ControllerServiceDAO+) && "
-            + "execution(org.apache.nifi.controller.service.ControllerServiceReference updateControllerServiceReferencingComponents(java.lang.String, boolean))")
+            + "execution(org.apache.nifi.controller.service.ControllerServiceReference updateControllerServiceReferencingComponents(java.lang.String, org.apache.nifi.controller.ScheduledState, org.apache.nifi.controller.service.ControllerServiceState))")
     public Object updateControllerServiceReferenceAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
         // update the controller service references
         final ControllerServiceReference controllerServiceReference = (ControllerServiceReference) proceedingJoinPoint.proceed();
@@ -271,7 +270,7 @@ public class ControllerServiceAuditor extends NiFiAuditor {
                     serviceAction.setSourceName(controllerService.getName());
                     serviceAction.setSourceType(Component.ControllerService);
                     serviceAction.setComponentDetails(serviceDetails);
-                    serviceAction.setOperation(controllerService.isDisabled() ? Operation.Disable : Operation.Enable);
+                    serviceAction.setOperation(isDisabled(controllerService) ? Operation.Disable : Operation.Enable);
                     actions.add(serviceAction);
                     
                     // need to consider components referencing this controller service (transitive)
@@ -324,7 +323,7 @@ public class ControllerServiceAuditor extends NiFiAuditor {
      * @param operation
      * @return
      */
-    public Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation) {
+    private Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation) {
         return generateAuditRecord(controllerService, operation, null);
     }
 
@@ -336,7 +335,7 @@ public class ControllerServiceAuditor extends NiFiAuditor {
      * @param actionDetails
      * @return
      */
-    public Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation, ActionDetails actionDetails) {
+    private Action generateAuditRecord(ControllerServiceNode controllerService, Operation operation, ActionDetails actionDetails) {
         Action action = null;
 
         // get the current user
@@ -384,9 +383,6 @@ public class ControllerServiceAuditor extends NiFiAuditor {
         if (controllerServiceDTO.getAnnotationData() != null) {
             values.put(ANNOTATION_DATA, controllerService.getAnnotationData());
         }
-        if (controllerServiceDTO.getAnnotationData() != null) {
-            values.put(AVAILABILITY, controllerService.getAnnotationData());
-        }
         if (controllerServiceDTO.getProperties() != null) {
             // for each property specified, extract its configured value
             Map<String, String> properties = controllerServiceDTO.getProperties();
@@ -428,4 +424,13 @@ public class ControllerServiceAuditor extends NiFiAuditor {
         return specDescriptor;
     }
 
+    /**
+     * Returns whether the specified controller service is disabled (or disabling).
+     * 
+     * @param controllerService
+     * @return 
+     */
+    private boolean isDisabled(final ControllerServiceNode controllerService) {
+        return ControllerServiceState.DISABLED.equals(controllerService.getState()) || ControllerServiceState.DISABLING.equals(controllerService.getState());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
new file mode 100644
index 0000000..ba2cdfb
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/audit/ReportingTaskAuditor.java
@@ -0,0 +1,353 @@
+/*
+ * 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.audit;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.nifi.action.Action;
+import org.apache.nifi.action.Component;
+import org.apache.nifi.action.Operation;
+import org.apache.nifi.action.component.details.ExtensionDetails;
+import org.apache.nifi.action.details.ActionDetails;
+import org.apache.nifi.action.details.ConfigureDetails;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ReportingTaskNode;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.web.security.user.NiFiUserUtils;
+import org.apache.nifi.user.NiFiUser;
+import org.apache.nifi.web.api.dto.ReportingTaskDTO;
+import org.apache.nifi.web.dao.ReportingTaskDAO;
+import org.aspectj.lang.ProceedingJoinPoint;
+import org.aspectj.lang.annotation.Around;
+import org.aspectj.lang.annotation.Aspect;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Audits reporting creation/removal and configuration changes.
+ */
+@Aspect
+public class ReportingTaskAuditor extends NiFiAuditor {
+
+    private static final Logger logger = LoggerFactory.getLogger(ReportingTaskAuditor.class);
+
+    private static final String NAME = "Name";
+    private static final String ANNOTATION_DATA = "Annotation Data";
+
+    /**
+     * Audits the creation of reporting task via createReportingTask().
+     *
+     * This method only needs to be run 'after returning'. However, in Java 7
+     * the order in which these methods are returned from
+     * Class.getDeclaredMethods (even though there is no order guaranteed) seems
+     * to differ from Java 6. SpringAOP depends on this ordering to determine
+     * advice precedence. By normalizing all advice into Around advice we can
+     * alleviate this issue.
+     *
+     * @param proceedingJoinPoint
+     * @return
+     * @throws java.lang.Throwable
+     */
+    @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && "
+            + "execution(org.apache.nifi.controller.ReportingTaskNode createReportingTask(org.apache.nifi.web.api.dto.ReportingTaskDTO))")
+    public ReportingTaskNode createReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint) throws Throwable {
+        // update the reporting task state
+        ReportingTaskNode reportingTask = (ReportingTaskNode) proceedingJoinPoint.proceed();
+
+        // if no exceptions were thrown, add the reporting task action...
+        final Action action = generateAuditRecord(reportingTask, Operation.Add);
+
+        // save the actions
+        if (action != null) {
+            saveAction(action, logger);
+        }
+
+        return reportingTask;
+    }
+
+    /**
+     * Audits the configuration of a reporting task.
+     *
+     * @param proceedingJoinPoint
+     * @param reportingTaskDTO
+     * @param reportingTaskDAO
+     * @return
+     * @throws Throwable
+     */
+    @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && "
+            + "execution(org.apache.nifi.controller.ReportingTaskNode updateReportingTask(org.apache.nifi.web.api.dto.ReportingTaskDTO)) && "
+            + "args(reportingTaskDTO) && "
+            + "target(reportingTaskDAO)")
+    public Object updateReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint, ReportingTaskDTO reportingTaskDTO, ReportingTaskDAO reportingTaskDAO) throws Throwable {
+        // determine the initial values for each property/setting thats changing
+        ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskDTO.getId());
+        final Map<String, String> values = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO);
+        final ScheduledState scheduledState = reportingTask.getScheduledState();
+
+        // update the reporting task state
+        final ReportingTaskNode updatedReportingTask = (ReportingTaskNode) proceedingJoinPoint.proceed();
+
+        // if no exceptions were thrown, add the reporting task action...
+        reportingTask = reportingTaskDAO.getReportingTask(updatedReportingTask.getIdentifier());
+
+        // get the current user
+        final NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+        // ensure the user was found
+        if (user != null) {
+            // determine the updated values
+            Map<String, String> updatedValues = extractConfiguredPropertyValues(reportingTask, reportingTaskDTO);
+
+            // create the reporting task details
+            ExtensionDetails taskDetails = new ExtensionDetails();
+            taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName());
+
+            // create a reporting task action
+            Date actionTimestamp = new Date();
+            Collection<Action> actions = new ArrayList<>();
+
+            // go through each updated value
+            for (String property : updatedValues.keySet()) {
+                String newValue = updatedValues.get(property);
+                String oldValue = values.get(property);
+                Operation operation = null;
+
+                // determine the type of operation
+                if (oldValue == null || newValue == null || !newValue.equals(oldValue)) {
+                    operation = Operation.Configure;
+                }
+
+                // create a configuration action accordingly
+                if (operation != null) {
+                    // clear the value if this property is sensitive
+                    final PropertyDescriptor propertyDescriptor = reportingTask.getReportingTask().getPropertyDescriptor(property);
+                    if (propertyDescriptor != null && propertyDescriptor.isSensitive()) {
+                        if (newValue != null) {
+                            newValue = "********";
+                        }
+                        if (oldValue != null) {
+                            oldValue = "********";
+                        }
+                    } else if (ANNOTATION_DATA.equals(property)) {
+                        if (newValue != null) {
+                            newValue = "<annotation data not shown>";
+                        }
+                        if (oldValue != null) {
+                            oldValue = "<annotation data not shown>";
+                        }
+                    }
+
+                    final ConfigureDetails actionDetails = new ConfigureDetails();
+                    actionDetails.setName(property);
+                    actionDetails.setValue(newValue);
+                    actionDetails.setPreviousValue(oldValue);
+
+                    // create a configuration action
+                    Action configurationAction = new Action();
+                    configurationAction.setUserDn(user.getDn());
+                    configurationAction.setUserName(user.getUserName());
+                    configurationAction.setOperation(operation);
+                    configurationAction.setTimestamp(actionTimestamp);
+                    configurationAction.setSourceId(reportingTask.getIdentifier());
+                    configurationAction.setSourceName(reportingTask.getName());
+                    configurationAction.setSourceType(Component.ReportingTask);
+                    configurationAction.setComponentDetails(taskDetails);
+                    configurationAction.setActionDetails(actionDetails);
+                    actions.add(configurationAction);
+                }
+            }
+
+            // determine the new executing state
+            final ScheduledState updatedScheduledState = reportingTask.getScheduledState();
+
+            // determine if the running state has changed and its not disabled
+            if (scheduledState != updatedScheduledState) {
+                // create a reporting task action
+                Action taskAction = new Action();
+                taskAction.setUserDn(user.getDn());
+                taskAction.setUserName(user.getUserName());
+                taskAction.setTimestamp(new Date());
+                taskAction.setSourceId(reportingTask.getIdentifier());
+                taskAction.setSourceName(reportingTask.getName());
+                taskAction.setSourceType(Component.ReportingTask);
+                taskAction.setComponentDetails(taskDetails);
+
+                // set the operation accordingly
+                if (ScheduledState.RUNNING.equals(updatedScheduledState)) {
+                    taskAction.setOperation(Operation.Start);
+                } else if (ScheduledState.DISABLED.equals(updatedScheduledState)) {
+                    taskAction.setOperation(Operation.Disable);
+                } else {
+                    // state is now stopped... consider the previous state
+                    if (ScheduledState.RUNNING.equals(scheduledState)) {
+                        taskAction.setOperation(Operation.Stop);
+                    } else if (ScheduledState.DISABLED.equals(scheduledState)) {
+                        taskAction.setOperation(Operation.Enable);
+                    }
+                }
+                actions.add(taskAction);
+            }
+
+            // ensure there are actions to record
+            if (!actions.isEmpty()) {
+                // save the actions
+                saveActions(actions, logger);
+            }
+        }
+
+        return updatedReportingTask;
+    }
+
+    /**
+     * Audits the removal of a reporting task via deleteReportingTask().
+     *
+     * @param proceedingJoinPoint
+     * @param reportingTaskId
+     * @param reportingTaskDAO
+     * @throws Throwable
+     */
+    @Around("within(org.apache.nifi.web.dao.ReportingTaskDAO+) && "
+            + "execution(void deleteReportingTask(java.lang.String)) && "
+            + "args(reportingTaskId) && "
+            + "target(reportingTaskDAO)")
+    public void removeReportingTaskAdvice(ProceedingJoinPoint proceedingJoinPoint, String reportingTaskId, ReportingTaskDAO reportingTaskDAO) throws Throwable {
+        // get the reporting task before removing it
+        ReportingTaskNode reportingTask = reportingTaskDAO.getReportingTask(reportingTaskId);
+
+        // remove the reporting task
+        proceedingJoinPoint.proceed();
+
+        // if no exceptions were thrown, add removal actions...
+        // audit the reporting task removal
+        final Action action = generateAuditRecord(reportingTask, Operation.Remove);
+
+        // save the actions
+        if (action != null) {
+            saveAction(action, logger);
+        }
+    }
+
+    /**
+     * Generates an audit record for the creation of a reporting task.
+     *
+     * @param reportingTask
+     * @param operation
+     * @return
+     */
+    public Action generateAuditRecord(ReportingTaskNode reportingTask, Operation operation) {
+        return generateAuditRecord(reportingTask, operation, null);
+    }
+
+    /**
+     * Generates an audit record for the creation of a reporting task.
+     *
+     * @param reportingTask
+     * @param operation
+     * @param actionDetails
+     * @return
+     */
+    public Action generateAuditRecord(ReportingTaskNode reportingTask, Operation operation, ActionDetails actionDetails) {
+        Action action = null;
+
+        // get the current user
+        NiFiUser user = NiFiUserUtils.getNiFiUser();
+
+        // ensure the user was found
+        if (user != null) {
+            // create the reporting task details
+            ExtensionDetails taskDetails = new ExtensionDetails();
+            taskDetails.setType(reportingTask.getReportingTask().getClass().getSimpleName());
+
+            // create the reporting task action for adding this reporting task
+            action = new Action();
+            action.setUserDn(user.getDn());
+            action.setUserName(user.getUserName());
+            action.setOperation(operation);
+            action.setTimestamp(new Date());
+            action.setSourceId(reportingTask.getIdentifier());
+            action.setSourceName(reportingTask.getName());
+            action.setSourceType(Component.ReportingTask);
+            action.setComponentDetails(taskDetails);
+
+            if (actionDetails != null) {
+                action.setActionDetails(actionDetails);
+            }
+        }
+
+        return action;
+    }
+
+    /**
+     * Extracts the values for the configured properties from the specified ReportingTask.
+     *
+     * @param reportingTask
+     * @param reportingTaskDTO
+     * @return
+     */
+    private Map<String, String> extractConfiguredPropertyValues(ReportingTaskNode reportingTask, ReportingTaskDTO reportingTaskDTO) {
+        Map<String, String> values = new HashMap<>();
+
+        if (reportingTaskDTO.getName() != null) {
+            values.put(NAME, reportingTask.getName());
+        }
+        if (reportingTaskDTO.getAnnotationData() != null) {
+            values.put(ANNOTATION_DATA, reportingTask.getAnnotationData());
+        }
+        if (reportingTaskDTO.getProperties() != null) {
+            // for each property specified, extract its configured value
+            Map<String, String> properties = reportingTaskDTO.getProperties();
+            Map<PropertyDescriptor, String> configuredProperties = reportingTask.getProperties();
+            for (String propertyName : properties.keySet()) {
+                // build a descriptor for getting the configured value
+                PropertyDescriptor propertyDescriptor = new PropertyDescriptor.Builder().name(propertyName).build();
+                String configuredPropertyValue = configuredProperties.get(propertyDescriptor);
+
+                // if the configured value couldn't be found, use the default value from the actual descriptor
+                if (configuredPropertyValue == null) {
+                    propertyDescriptor = locatePropertyDescriptor(configuredProperties.keySet(), propertyDescriptor);
+                    configuredPropertyValue = propertyDescriptor.getDefaultValue();
+                }
+                values.put(propertyName, configuredPropertyValue);
+            }
+        }
+
+        return values;
+    }
+
+    /**
+     * Locates the actual property descriptor for the given spec property
+     * descriptor.
+     *
+     * @param propertyDescriptors
+     * @param specDescriptor
+     * @return
+     */
+    private PropertyDescriptor locatePropertyDescriptor(Set<PropertyDescriptor> propertyDescriptors, PropertyDescriptor specDescriptor) {
+        for (PropertyDescriptor propertyDescriptor : propertyDescriptors) {
+            if (propertyDescriptor.equals(specDescriptor)) {
+                return propertyDescriptor;
+            }
+        }
+        return specDescriptor;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.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/NiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
index 0f53de7..02b97a0 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
@@ -982,11 +982,11 @@ public interface NiFiServiceFacade {
      * 
      * @param revision
      * @param controllerServiceId
-     * @param enabled
-     * @param state the value of state 
+     * @param scheduledState
+     * @param controllerServiceState the value of state 
      * @return the org.apache.nifi.web.ConfigurationSnapshot<java.util.Set<org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO>> 
      */
-    ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> updateControllerServiceReferencingComponents(Revision revision, String controllerServiceId, Boolean enabled, String state);
+    ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> updateControllerServiceReferencingComponents(Revision revision, String controllerServiceId, org.apache.nifi.controller.ScheduledState scheduledState, org.apache.nifi.controller.service.ControllerServiceState controllerServiceState);
     
     /**
      * Updates the specified label.
@@ -1012,7 +1012,7 @@ public interface NiFiServiceFacade {
      * @param controllerServiceDTO
      */
     void verifyUpdateControllerService(ControllerServiceDTO controllerServiceDTO);
-
+    
     /**
      * Verifies the specified controller service can be removed.
      *

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.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/StandardNiFiServiceFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
index c16ea48..ec288d0 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
@@ -1228,11 +1228,11 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     @Override
-    public ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> updateControllerServiceReferencingComponents(final Revision revision, final String controllerServiceId, final Boolean enabled, final String state) {
+    public ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> updateControllerServiceReferencingComponents(final Revision revision, final String controllerServiceId, final org.apache.nifi.controller.ScheduledState scheduledState, final org.apache.nifi.controller.service.ControllerServiceState controllerServiceState) {
         return optimisticLockingManager.configureFlow(revision, new ConfigurationRequest<Set<ControllerServiceReferencingComponentDTO>>() {
             @Override
             public Set<ControllerServiceReferencingComponentDTO> execute() {
-                final ControllerServiceReference reference = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, enabled, state);
+                final ControllerServiceReference reference = controllerServiceDAO.updateControllerServiceReferencingComponents(controllerServiceId, scheduledState, controllerServiceState);
                 return dtoFactory.createControllerServiceReferencingComponentsDto(reference);
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.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/api/ControllerServiceResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
index ac1805c..732dec8 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ControllerServiceResource.java
@@ -51,6 +51,8 @@ import org.apache.nifi.web.api.dto.RevisionDTO;
 import org.apache.nifi.web.api.request.ClientIdParameter;
 import org.apache.nifi.web.api.request.LongParameter;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
 import org.apache.nifi.web.api.entity.ControllerServiceEntity;
@@ -385,10 +387,9 @@ public class ControllerServiceResource extends ApplicationResource {
      * @param availability Whether the controller service is available on the NCM only (ncm) or on the 
      * nodes only (node). If this instance is not clustered all services should use the node availability.
      * @param id The id of the controller service to retrieve
-     * @param state Sets the state of referencing scheduled components (Processors and Reporting Tasks). 
-     * Cannot be set in conjunction with enabled.
-     * @param enabled Sets the state of referencing controller services. Cannot be set in conjunction 
-     * with state
+     * @param state Sets the state of referencing components. A value of RUNNING or STOPPED will update
+     * referencing schedulable components (Processors and Reporting Tasks). A value of ENABLED or
+     * DISABLED will update referencing controller services.
      * @return A controllerServiceEntity.
      */
     @PUT
@@ -402,18 +403,41 @@ public class ControllerServiceResource extends ApplicationResource {
             @FormParam(VERSION) LongParameter version,
             @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
             @PathParam("availability") String availability, @PathParam("id") String id,
-            @FormParam("state") String state, @FormParam("enabled") Boolean enabled) {
+            @FormParam("state") String state) {
 
+        // parse the state to determine the desired action
+        ScheduledState scheduledState = null;
+        try {
+            scheduledState = ScheduledState.valueOf(state);
+        } catch (final IllegalArgumentException iae) {
+            // ignore
+        }
+        
+        ControllerServiceState controllerServiceState = null;
+        try {
+            controllerServiceState = ControllerServiceState.valueOf(state);
+        } catch (final IllegalArgumentException iae) {
+            // ignore
+        }
+        
         // ensure an action has been specified
-        if (state == null && enabled == null) {
-            throw new IllegalArgumentException("Must specify whether updating the state of Processors and Reporting Tasks or Controller Services.");
+        if (scheduledState == null && controllerServiceState == null) {
+            throw new IllegalArgumentException("Must specify whether updating the state a valid state. To update referencing Processors "
+                    + "and Reporting Tasks the state should be RUNNING or STOPPED. To update the referencing Controller Services the "
+                    + "state should be ENABLED or DISABLED.");
+        }
+        
+        // ensure the scheduled state is not disabled
+        if (scheduledState != null && ScheduledState.DISABLED.equals(scheduledState)) {
+            throw new IllegalArgumentException("Cannot disable referencing components.");
         }
         
-        // ensure both actions are not specified at the same time
-        if (state != null && enabled != null) {
-            throw new IllegalArgumentException("Cannot specify the state of Processors and Reporting Tasks and Controller Services in the same request");
+        // ensure the controller service state is not ENABLING or DISABLING
+        if (controllerServiceState != null && (ControllerServiceState.ENABLING.equals(controllerServiceState) || ControllerServiceState.DISABLING.equals(controllerServiceState))) {
+            throw new IllegalArgumentException("Cannot set the referencing services to ENABLING or DISABLING");
         }
         
+        // determine the availability
         final Availability avail = parseAvailability(availability);
         
         // replicate if cluster manager
@@ -424,6 +448,7 @@ public class ControllerServiceResource extends ApplicationResource {
         // handle expects request (usually from the cluster manager)
         final String expects = httpServletRequest.getHeader(WebClusterManager.NCM_EXPECTS_HTTP_HEADER);
         if (expects != null) {
+//            serviceFacade.verifyUpdateControllerServiceReferences(true);
             return generateContinueResponse().build();
         }
         
@@ -435,7 +460,7 @@ public class ControllerServiceResource extends ApplicationResource {
         
         // get the controller service
         final ConfigurationSnapshot<Set<ControllerServiceReferencingComponentDTO>> response = 
-                serviceFacade.updateControllerServiceReferencingComponents(new Revision(clientVersion, clientId.getClientId()), id, enabled, state);
+                serviceFacade.updateControllerServiceReferencingComponents(new Revision(clientVersion, clientId.getClientId()), id, scheduledState, controllerServiceState);
 
         // create the revision
         final RevisionDTO revision = new RevisionDTO();
@@ -465,7 +490,7 @@ public class ControllerServiceResource extends ApplicationResource {
      * @param name The name of the controller service
      * @param annotationData The annotation data for the controller service
      * @param comments The comments for the controller service
-     * @param enabled Whether this controller service is enabled or not
+     * @param state The state of this controller service. Should be ENABLED or DISABLED.
      * @param markedForDeletion Array of property names whose value should be removed.
      * @param formParams Additionally, the processor properties and styles are
      * specified in the form parameters. Because the property names and styles
@@ -495,7 +520,7 @@ public class ControllerServiceResource extends ApplicationResource {
             @FormParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId,
             @PathParam("availability") String availability, @PathParam("id") String id, @FormParam("name") String name,
             @FormParam("annotationData") String annotationData, @FormParam("comments") String comments,
-            @FormParam("enabled") Boolean enabled, @FormParam("markedForDeletion[]") List<String> markedForDeletion,
+            @FormParam("state") String state, @FormParam("markedForDeletion[]") List<String> markedForDeletion,
             MultivaluedMap<String, String> formParams) {
 
         // create collections for holding the controller service properties
@@ -528,7 +553,7 @@ public class ControllerServiceResource extends ApplicationResource {
         controllerServiceDTO.setName(name);
         controllerServiceDTO.setAnnotationData(annotationData);
         controllerServiceDTO.setComments(comments);
-        controllerServiceDTO.setEnabled(enabled);
+        controllerServiceDTO.setState(state);
 
         // only set the properties when appropriate
         if (!updatedProperties.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.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/api/dto/DtoFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
index e82bcbd..47c8161 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/dto/DtoFactory.java
@@ -97,7 +97,6 @@ import org.apache.nifi.scheduling.SchedulingStrategy;
 import org.apache.nifi.user.NiFiUser;
 import org.apache.nifi.user.NiFiUserGroup;
 import org.apache.nifi.util.FormatUtils;
-import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.Revision;
 import org.apache.nifi.web.api.dto.PropertyDescriptorDTO.AllowableValueDTO;
 import org.apache.nifi.web.api.dto.action.ActionDTO;
@@ -853,7 +852,7 @@ public final class DtoFactory {
         dto.setId(controllerServiceNode.getIdentifier());
         dto.setName(controllerServiceNode.getName());
         dto.setType(controllerServiceNode.getControllerServiceImplementation().getClass().getName());
-        dto.setEnabled(!controllerServiceNode.isDisabled());
+        dto.setState(controllerServiceNode.getState().name());
         dto.setComments(controllerServiceNode.getComments());
         
         // sort a copy of the properties
@@ -930,7 +929,7 @@ public final class DtoFactory {
                 dto.setReferenceType(Processor.class.getSimpleName());
             } else if (component instanceof ControllerServiceNode) {
                 final ControllerServiceNode node = ((ControllerServiceNode) component);
-                dto.setEnabled(!node.isDisabled());
+                dto.setState(node.getState().name());
                 dto.setType(node.getControllerServiceImplementation().getClass().getName());
                 dto.setReferenceType(ControllerService.class.getSimpleName());
             } else if (component instanceof ReportingTask) {
@@ -2386,8 +2385,6 @@ public final class DtoFactory {
     }
 
     /* setters */
-    public void setProperties(NiFiProperties properties) {
-    }
 
     public void setControllerServiceLookup(ControllerServiceLookup lookup) {
         this.controllerServiceLookup = lookup;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.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/dao/ControllerServiceDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
index cec4b92..f613d9c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/ControllerServiceDAO.java
@@ -17,9 +17,11 @@
 package org.apache.nifi.web.dao;
 
 import java.util.Set;
+import org.apache.nifi.controller.ScheduledState;
 
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceReference;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 
 /**
@@ -70,11 +72,11 @@ public interface ControllerServiceDAO {
      * Updates the referencing components for the specified controller service.
      * 
      * @param controllerServiceId
-     * @param enabled
-     * @param state the value of state 
+     * @param scheduledState
+     * @param controllerServiceState the value of state 
      * @return the org.apache.nifi.controller.service.ControllerServiceReference 
      */
-    ControllerServiceReference updateControllerServiceReferencingComponents(String controllerServiceId, Boolean enabled, String state);
+    ControllerServiceReference updateControllerServiceReferencingComponents(String controllerServiceId, ScheduledState scheduledState, ControllerServiceState controllerServiceState);
     
     /**
      * Determines whether this controller service can be updated.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.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/dao/impl/StandardControllerServiceDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
index 9c2f0e0..a9a63ad 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardControllerServiceDAO.java
@@ -26,6 +26,7 @@ import org.apache.nifi.controller.exception.ValidationException;
 import org.apache.nifi.controller.service.ControllerServiceNode;
 import org.apache.nifi.controller.service.ControllerServiceProvider;
 import org.apache.nifi.controller.service.ControllerServiceReference;
+import org.apache.nifi.controller.service.ControllerServiceState;
 import org.apache.nifi.web.ResourceNotFoundException;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.dao.ControllerServiceDAO;
@@ -122,15 +123,13 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
         configureControllerService(controllerService, controllerServiceDTO);
 
         // enable or disable as appropriate
-        if (isNotNull(controllerServiceDTO.getEnabled())) {
-            final boolean proposedDisabled = !controllerServiceDTO.getEnabled();
-            
-            if (proposedDisabled != controllerService.isDisabled()) {
-                if (proposedDisabled) {
-                    serviceProvider.disableControllerService(controllerService);
-                } else {
-                    serviceProvider.enableControllerService(controllerService);
-                }
+        if (isNotNull(controllerServiceDTO.getState())) {
+            final ControllerServiceState controllerServiceState = ControllerServiceState.valueOf(controllerServiceDTO.getState());
+
+            if (ControllerServiceState.ENABLED.equals(controllerServiceState)) {
+                serviceProvider.enableControllerService(controllerService);
+            } else if (ControllerServiceState.DISABLED.equals(controllerServiceState)) {
+                serviceProvider.disableControllerService(controllerService);
             }
         }
         
@@ -138,36 +137,22 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
     }
 
     @Override
-    public ControllerServiceReference updateControllerServiceReferencingComponents(final String controllerServiceId, final Boolean enabled, String state) {
+    public ControllerServiceReference updateControllerServiceReferencingComponents(final String controllerServiceId, final ScheduledState scheduledState, final ControllerServiceState controllerServiceState) {
         // get the controller service
         final ControllerServiceNode controllerService = locateControllerService(controllerServiceId);
         
-        // TODO - these actions need to be atomic... can't have partial success... maybe already handled?
-        
-        if (enabled != null) {
-            if (enabled) {
+        // this request is either acting upon referncing services or schedulable components
+        if (controllerServiceState != null) {
+            if (ControllerServiceState.ENABLED.equals(controllerServiceState)) {
                 serviceProvider.enableReferencingServices(controllerService);
             } else {
                 serviceProvider.disableReferencingServices(controllerService);
             }
-        } else if (state != null) {
-            try {
-                final ScheduledState scheduledState = ScheduledState.valueOf(state);
-                
-                switch (scheduledState) {
-                    case RUNNING:
-                        serviceProvider.scheduleReferencingComponents(controllerService);
-                        break;
-                    case STOPPED:
-                        serviceProvider.unscheduleReferencingComponents(controllerService);
-                        break;
-                    default:
-                        throw new IllegalArgumentException(String.format(
-                                "The specified state (%s) is not valid. Valid options are 'RUNNING' and 'STOPPED'.", state));
-                }
-            } catch (IllegalArgumentException iae) {
-                throw new IllegalArgumentException(String.format(
-                        "The specified state (%s) is not valid. Valid options are 'RUNNING' and 'STOPPED'.", state));
+        } else if (scheduledState != null) {
+            if (ScheduledState.RUNNING.equals(scheduledState)) {
+                serviceProvider.scheduleReferencingComponents(controllerService);
+            } else {
+                serviceProvider.unscheduleReferencingComponents(controllerService);
             }
         }
         
@@ -205,11 +190,24 @@ public class StandardControllerServiceDAO extends ComponentDAO implements Contro
      * @param controllerServiceDTO 
      */
     private void verifyUpdate(final ControllerServiceNode controllerService, final ControllerServiceDTO controllerServiceDTO) {
-        if (isNotNull(controllerServiceDTO.getEnabled())) {
-            if (controllerServiceDTO.getEnabled()) {
-                controllerService.verifyCanEnable();
-            } else {
-                controllerService.verifyCanDisable();
+        // validate the new controller service state if appropriate
+        if (isNotNull(controllerServiceDTO.getState())) {
+            try {
+                // attempt to parse the service state
+                final ControllerServiceState controllerServiceState = ControllerServiceState.valueOf(controllerServiceDTO.getState());
+                
+                // ensure the state is valid
+                if (ControllerServiceState.ENABLING.equals(controllerServiceState) || ControllerServiceState.DISABLING.equals(controllerServiceState)) {
+                    throw new IllegalArgumentException();
+                }
+                
+                if (ControllerServiceState.ENABLED.equals(controllerServiceState)) {
+                    controllerService.verifyCanEnable();
+                } else if (ControllerServiceState.DISABLED.equals(controllerServiceState)) {
+                    controllerService.verifyCanDisable();
+                }
+            } catch (IllegalArgumentException iae) {
+                throw new IllegalArgumentException("Controller Service state: Value must be one of [ENABLED, DISABLED]");
             }
         }
         

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.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/dao/impl/StandardReportingTaskDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
index 5cccb81..793f77d 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/dao/impl/StandardReportingTaskDAO.java
@@ -17,6 +17,7 @@
 package org.apache.nifi.web.dao.impl;
 
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -105,7 +106,7 @@ public class StandardReportingTaskDAO extends ComponentDAO implements ReportingT
      */
     @Override
     public Set<ReportingTaskNode> getReportingTasks() {
-        return null;
+        return new HashSet<>();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
index 8a12352..134199b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
@@ -35,8 +35,7 @@
 
     <!-- dto factory -->
     <bean id="dtoFactory" class="org.apache.nifi.web.api.dto.DtoFactory">
-        <property name="properties" ref="nifiProperties"/>
-        <property name="controllerServiceLookup" ref="flowController" />
+        <property name="controllerServiceLookup" ref="controllerServiceProvider" />
     </bean>
 
     <!-- snippet utils -->
@@ -73,6 +72,9 @@
     <bean id="controllerServiceDAO" class="org.apache.nifi.web.dao.impl.StandardControllerServiceDAO">
         <property name="serviceProvider" ref="controllerServiceProvider"/>
     </bean>
+    <bean id="reportingTaskDAO" class="org.apache.nifi.web.dao.impl.StandardReportingTaskDAO">
+        <property name="flowController" ref="flowController"/>
+    </bean>
     <bean id="templateDAO" class="org.apache.nifi.web.dao.impl.StandardTemplateDAO">
         <property name="flowController" ref="flowController"/>
         <property name="snippetUtils" ref="snippetUtils"/>
@@ -137,6 +139,11 @@
         <property name="properties" ref="nifiProperties"/>
         <property name="clusterManager" ref="clusterManager"/>
     </bean>
+    <bean id="reportingTaskResource" class="org.apache.nifi.web.api.ReportingTaskResource" scope="singleton">
+        <property name="serviceFacade" ref="serviceFacade"/>
+        <property name="properties" ref="nifiProperties"/>
+        <property name="clusterManager" ref="clusterManager"/>
+    </bean>
     <bean id="processGroupResource" class="org.apache.nifi.web.api.ProcessGroupResource" scope="prototype">
         <property name="serviceFacade" ref="serviceFacade"/>
         <property name="properties" ref="nifiProperties"/>
@@ -312,6 +319,11 @@
         <property name="auditService" ref="auditService"/>
         <property name="processGroupDAO" ref="processGroupDAO"/>
     </bean>
+    <bean id="reportingTaskAuditor" class="org.apache.nifi.audit.ReportingTaskAuditor">
+        <property name="serviceFacade" ref="serviceFacade"/>
+        <property name="auditService" ref="auditService"/>
+        <property name="processGroupDAO" ref="processGroupDAO"/>
+    </bean>
     
     <!-- NiFi locking -->
     <bean id="serviceFacadeLock" class="org.apache.nifi.web.NiFiServiceFacadeLock"/>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
index 0e2a292..68f67a0 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/pom.xml
@@ -419,6 +419,7 @@
                                                 <include>${staging.dir}/css/dialog.css</include>
                                                 <include>${staging.dir}/css/new-processor-dialog.css</include>
                                                 <include>${staging.dir}/css/new-controller-service-dialog.css</include>
+                                                <include>${staging.dir}/css/new-reporting-task-dialog.css</include>
                                                 <include>${staging.dir}/css/graph.css</include>
                                                 <include>${staging.dir}/css/header.css</include>
                                                 <include>${staging.dir}/css/main.css</include>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
index 3f361ce..77f41bd 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/canvas.jsp
@@ -83,6 +83,7 @@
         <jsp:include page="/WEB-INF/partials/canvas/disable-controller-service-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/enable-controller-service-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/new-controller-service-dialog.jsp"/>
+        <jsp:include page="/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/new-processor-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/new-port-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/canvas/new-process-group-dialog.jsp"/>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp
new file mode 100644
index 0000000..cfb3992
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/canvas/new-reporting-task-dialog.jsp
@@ -0,0 +1,53 @@
+<%--
+ 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.
+--%>
+<%@ page contentType="text/html" pageEncoding="UTF-8" session="false" %>
+<div id="new-reporting-task-dialog">
+    <div class="dialog-content">
+        <div id="reporting-task-type-filter-controls">
+            <div id="controller-service-type-filter-container">
+                <input type="text" id="reporting-task-type-filter"/>
+                <div id="reporting-task-type-filter-options"></div>
+            </div>
+            <div id="reporting-task-type-filter-status">
+                Displaying&nbsp;<span id="displayed-reporting-task-types"></span>&nbsp;of&nbsp;<span id="total-reporting-task-types"></span>
+            </div>
+        </div>
+        <div id="reporting-task-tag-cloud-container">
+            <div class="setting">
+                <div class="setting-name">Tags</div>
+                <div class="setting-field">
+                    <div id="reporting-task-tag-cloud"></div>
+                </div>
+            </div>
+        </div>
+        <div id="reporting-task-types-container">
+            <div id="reporting-task-types-table" class="unselectable"></div>
+            <div id="reporting-task-description-container" class="hidden">
+                <div id="reporting-task-type-name" class="ellipsis"></div>
+                <div id="reporting-task-type-description" class="ellipsis multiline"></div>
+                <span class="hidden" id="selected-reporting-task-name"></span>
+                <span class="hidden" id="selected-reporting-task-type"></span>
+            </div>
+        </div>
+        <div class="clear"></div>
+        <div id="reporting-task-availability-container" class="hidden">
+            <div class="setting-name availability-label">Available on</div>
+            <div id="reporting-task-availability-combo"></div>
+            <div class="clear"></div>
+        </div>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css
index 9c31200..7d1852b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/canvas.css
@@ -31,6 +31,7 @@
 @import url(dialog.css);
 @import url(new-processor-dialog.css);
 @import url(new-controller-service-dialog.css);
+@import url(new-reporting-task-dialog.css);
 @import url(graph.css);
 @import url(header.css);
 @import url(main.css);

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css
new file mode 100644
index 0000000..24b4380
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/new-reporting-task-dialog.css
@@ -0,0 +1,152 @@
+/*
+ * 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.
+ */
+
+/* 
+    New controller service dialog.
+*/
+
+#new-reporting-task-dialog {
+    z-index: 1301;
+    display: none;
+    width: 800px;
+    height: 450px;
+    line-height: normal;
+}
+
+#reporting-task-tag-cloud-container {
+    float: left;
+    width: 190px;
+    height: 350px;
+}
+
+#reporting-task-types-container {
+    float: left;
+    width: 588px;
+    height: 350px;
+}
+
+#reporting-task-description-container {
+    width: 588px;
+    height: 100px;
+    margin-top: 10px;
+}
+
+#reporting-task-type-name {
+    font-size: 12pt;
+    font-weight: bold;
+    color: #527991;
+    margin-bottom: 8px;
+    width: 588px;
+}
+
+#reporting-task-type-description {
+    width: 588px;
+    height: 60px;
+}
+
+#reporting-task-availability-container {
+    position: relative;
+    left: 190px;
+    top: 9px;
+}
+
+div.availability-label {
+    float: left;
+    height: 24px;
+    line-height: 24px;
+}
+
+#reporting-task-availability-combo {
+    float: left;
+    margin-left: 6px;
+    width: 125px;
+    height: 18px;
+    line-height: 18px;
+}
+
+#reporting-task-types-table {
+    width: 586px;
+    height: 249px;
+    border: 1px solid #666;
+}
+
+#reporting-task-types-table-container th {
+    vertical-align: middle;
+}
+
+/*
+    Processor tag cloud
+*/
+
+#reporting-task-tag-cloud ul.tag-cloud {
+    max-height: 257px;
+}
+
+#reporting-task-tag-cloud ul.tag-cloud li {
+    max-width: 180px;
+}
+
+#reporting-task-tag-cloud div.tag-cloud-separator {
+    width: 180px;
+}
+
+#reporting-task-tag-cloud ul.tag-filter li {
+    width: 180px;
+}
+
+#reporting-task-tag-cloud div.selected-tag-text {
+    width: 160px;
+}
+
+/*
+    Processor type table filter
+*/
+
+#reporting-task-type-filter-controls {
+    float: right;
+    margin-top: -35px;
+    margin-right: 2px;
+    margin-bottom: 7px;
+}
+
+#reporting-task-type-filter-status {
+    font-size: 9px;
+    font-weight: bold;
+    color: #9f6000;
+    clear: left; 
+    line-height: normal;
+    margin-left: 5px;
+}
+
+#reporting-task-type-filter {
+    padding: 3px 0px 1px 3px;
+    font-size: 12px;
+    height: 18px;
+    line-height: 20px;
+    width: 173px;
+    border: 1px solid #ccc;
+    margin-right: 3px;
+    float: left;
+}
+
+#reporting-task-type-filter-options {
+    float: left;
+    height: 17px;
+    line-height: 17px;
+    width: 85px;
+    margin-top: 1px;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/573a8aa3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js
index dc59fa4..45d949a 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-controller-service.js
@@ -65,9 +65,7 @@ nf.ControllerService = (function () {
             return true;
         }
         
-        if ($('#controller-service-enabled').hasClass('checkbox-checked') && details['enabled'] === false) {
-            return true;
-        } else if ($('#controller-service-enabled').hasClass('checkbox-unchecked') && details['enabled'] === true) {
+        if ($('#controller-service-enabled').hasClass('checkbox-checked')) {
             return true;
         }
         
@@ -92,11 +90,9 @@ nf.ControllerService = (function () {
             controllerServiceDto['properties'] = properties;
         }
         
-        // mark the controller service disabled if appropriate
-        if ($('#controller-service-enabled').hasClass('checkbox-unchecked')) {
-            controllerServiceDto['enabled'] = false;
-        } else if ($('#controller-service-enabled').hasClass('checkbox-checked')) {
-            controllerServiceDto['enabled'] = true;
+        // mark the controller service enabled if appropriate
+        if ($('#controller-service-enabled').hasClass('checkbox-checked')) {
+            controllerServiceDto['state'] = 'ENABLED';
         }
 
         // create the controller service entity
@@ -281,7 +277,7 @@ nf.ControllerService = (function () {
                     updateReferencingComponentsBorder(referenceContainer);
                 });
                 
-                var serviceState = $('<div class="referencing-component-state"></div>').addClass(referencingComponent.enabled === true ? 'enabled' : 'disabled').addClass(referencingComponent.id + '-active-threads');
+                var serviceState = $('<div class="referencing-component-state"></div>').addClass(referencingComponent.state === 'ENABLED' ? 'enabled' : 'disabled').addClass(referencingComponent.id + '-active-threads');
                 var serviceId = $('<span class="referencing-service-id hidden"></span>').text(referencingComponent.id);
                 var serviceType = $('<span class="referencing-component-type"></span>').text(nf.Common.substringAfterLast(referencingComponent.type, '.'));
                 var serviceItem = $('<li></li>').append(serviceTwist).append(serviceState).append(serviceId).append(serviceLink).append(serviceType).append(referencingServiceReferencesContainer);
@@ -336,7 +332,7 @@ nf.ControllerService = (function () {
             data: {
                 clientId: revision.clientId,
                 version: revision.version,
-                enabled: enabled
+                state: enabled === true ? 'ENABLED' : 'DISABLED'
             },
             dataType: 'json'
         }).done(function (response) {
@@ -442,7 +438,7 @@ nf.ControllerService = (function () {
                     if (referencingComponent.referenceType === 'ControllerService') {
                         if (referencingComponent.enable === true) {
                             // update the current values for this component
-                            $(referencingComponent.id + '-state').removeClass('enabled disabled').addClass(referencingComponent.enabled === true ? 'enabled' : 'disabled');
+                            $(referencingComponent.id + '-state').removeClass('enabled disabled').addClass(referencingComponent.state === 'ENABLED' ? 'enabled' : 'disabled');
                             
                             // mark that something is still running
                             stillRunning = true;
@@ -566,7 +562,7 @@ nf.ControllerService = (function () {
 
                         // clear the tables
                         $('#controller-service-properties').propertytable('clear');
-
+                        
                         // removed the cached controller service details
                         $('#controller-service-configuration').removeData('controllerServiceDetails');
                     }
@@ -728,17 +724,11 @@ nf.ControllerService = (function () {
                 // record the controller service details
                 $('#controller-service-configuration').data('controllerServiceDetails', controllerService);
 
-                // determine if the enabled checkbox is checked or not
-                var controllerServiceEnableStyle = 'checkbox-checked';
-                if (controllerService['enabled'] === false) {
-                    controllerServiceEnableStyle = 'checkbox-unchecked';
-                }
-
                 // populate the controller service settings
                 $('#controller-service-id').text(controllerService['id']);
                 $('#controller-service-type').text(nf.Common.substringAfterLast(controllerService['type'], '.'));
                 $('#controller-service-name').val(controllerService['name']);
-                $('#controller-service-enabled').removeClass('checkbox-unchecked checkbox-checked').addClass(controllerServiceEnableStyle);
+                $('#controller-service-enabled').removeClass('checkbox-checked checkbox-unchecked').addClass('checkbox-unchecked');
                 $('#controller-service-comments').val(controllerService['comments']);
 
                 // select the availability when appropriate