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/07/16 16:47:12 UTC

[19/50] [abbrv] incubator-nifi git commit: NIFI-694: - Showing bulletins in the controller service and reporting task tables. - Fixed issue typo when emitting controller service bulletins.

NIFI-694:
- Showing bulletins in the controller service and reporting task tables.
- Fixed issue typo when emitting controller service bulletins.

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

Branch: refs/heads/master
Commit: 59aa8ffe100645498fdd4cea2214a203e3a3d433
Parents: 7a28903
Author: Matt Gilman <ma...@gmail.com>
Authored: Tue Jun 30 14:14:57 2015 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Tue Jun 30 14:14:57 2015 -0400

----------------------------------------------------------------------
 .../apache/nifi/reporting/BulletinQuery.java    |  12 ++
 .../web/api/dto/status/ControllerStatusDTO.java |  33 +++-
 .../nifi/events/VolatileBulletinRepository.java |  13 +-
 .../logging/ControllerServiceLogObserver.java   |   5 +-
 .../nifi/web/StandardNiFiServiceFacade.java     |  19 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  22 ++-
 .../nifi/web/controller/ControllerFacade.java   |  43 ++--
 .../src/main/webapp/css/controller-service.css  |   9 +-
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   |   3 +
 .../js/nf/canvas/nf-controller-service.js       |   9 +-
 .../src/main/webapp/js/nf/canvas/nf-settings.js | 194 +++++++++++++++++--
 11 files changed, 313 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinQuery.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinQuery.java b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinQuery.java
index cb5d7b3..7ba2089 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinQuery.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/reporting/BulletinQuery.java
@@ -23,6 +23,7 @@ import java.util.regex.Pattern;
  */
 public class BulletinQuery {
 
+    private final ComponentType sourceType;
     private final Pattern sourceIdPattern;
     private final Pattern groupIdPattern;
     private final Pattern namePattern;
@@ -31,6 +32,7 @@ public class BulletinQuery {
     private final Integer limit;
 
     private BulletinQuery(final Builder builder) {
+        this.sourceType = builder.sourceType;
         this.sourceIdPattern = builder.sourceIdPattern == null ? null : Pattern.compile(builder.sourceIdPattern);
         this.groupIdPattern = builder.groupIdPattern == null ? null : Pattern.compile(builder.groupIdPattern);
         this.namePattern = builder.namePattern == null ? null : Pattern.compile(builder.namePattern);
@@ -39,6 +41,10 @@ public class BulletinQuery {
         this.limit = builder.limit;
     }
 
+    public ComponentType getSourceType() {
+        return sourceType;
+    }
+
     public Pattern getSourceIdPattern() {
         return sourceIdPattern;
     }
@@ -65,6 +71,7 @@ public class BulletinQuery {
 
     public static class Builder {
 
+        private ComponentType sourceType;
         private String sourceIdPattern;
         private String groupIdPattern;
         private String namePattern;
@@ -87,6 +94,11 @@ public class BulletinQuery {
             return this;
         }
 
+        public Builder sourceType(ComponentType sourceType) {
+            this.sourceType = sourceType;
+            return this;
+        }
+
         public Builder nameMatches(String name) {
             this.namePattern = name;
             return this;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
index 7afc7bc..5d5eddf 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
@@ -40,6 +40,8 @@ public class ControllerStatusDTO {
     private Integer inactiveRemotePortCount;
 
     private List<BulletinDTO> bulletins;
+    private List<BulletinDTO> controllerServiceBulletins;
+    private List<BulletinDTO> reportingTaskBulletins;
 
     /**
      * The active thread count.
@@ -72,7 +74,8 @@ public class ControllerStatusDTO {
     }
 
     /**
-     * @return Used in clustering, will report the number of nodes connected vs the number of nodes in the cluster
+     * @return Used in clustering, will report the number of nodes connected vs
+     * the number of nodes in the cluster
      */
     @ApiModelProperty(
             value = "When clustered, reports the number of nodes connected vs the number of nodes in the cluster."
@@ -100,6 +103,34 @@ public class ControllerStatusDTO {
     }
 
     /**
+     * @return Controller service bulletins to be reported to the user
+     */
+    @ApiModelProperty(
+            value = "Controller service bulletins to be reported to the user."
+    )
+    public List<BulletinDTO> getControllerServiceBulletins() {
+        return controllerServiceBulletins;
+    }
+
+    public void setControllerServiceBulletins(List<BulletinDTO> controllerServiceBulletins) {
+        this.controllerServiceBulletins = controllerServiceBulletins;
+    }
+
+    /**
+     * @return Reporting task bulletins to be reported to the user
+     */
+    @ApiModelProperty(
+            value = "Reporting task bulletins to be reported to the user."
+    )
+    public List<BulletinDTO> getReportingTaskBulletins() {
+        return reportingTaskBulletins;
+    }
+
+    public void setReportingTaskBulletins(List<BulletinDTO> reportingTaskBulletins) {
+        this.reportingTaskBulletins = reportingTaskBulletins;
+    }
+
+    /**
      * @return whether or not there are pending user requests
      */
     @ApiModelProperty(

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
index 5172d34..c18fffd 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
@@ -102,6 +102,14 @@ public class VolatileBulletinRepository implements BulletinRepository {
                     }
                 }
 
+                // if a source component type was specified see if it should be excluded
+                if (bulletinQuery.getSourceType() != null) {
+                    // exclude if this bulletin source type doesn't match
+                    if (bulletin.getSourceType() == null || !bulletinQuery.getSourceType().equals(bulletin.getSourceType())) {
+                        return false;
+                    }
+                }
+
                 return true;
             }
         };
@@ -177,8 +185,9 @@ public class VolatileBulletinRepository implements BulletinRepository {
     }
 
     /**
-     * Overrides the default bulletin processing strategy. When a custom bulletin strategy is employed, bulletins will not be persisted in this repository and will sent to the specified strategy
-     * instead.
+     * Overrides the default bulletin processing strategy. When a custom
+     * bulletin strategy is employed, bulletins will not be persisted in this
+     * repository and will sent to the specified strategy instead.
      *
      * @param strategy bulletin strategy
      */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java
index 837e1c4..d9eaa12 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/logging/ControllerServiceLogObserver.java
@@ -24,6 +24,7 @@ import org.apache.nifi.reporting.ComponentType;
 import org.apache.nifi.reporting.Severity;
 
 public class ControllerServiceLogObserver implements LogObserver {
+
     private final BulletinRepository bulletinRepository;
     private final ControllerServiceNode serviceNode;
 
@@ -38,8 +39,8 @@ public class ControllerServiceLogObserver implements LogObserver {
         // the LogLevel is (INFO and ERROR map directly and all others we will just accept as they are).
         final String bulletinLevel = message.getLevel() == LogLevel.WARN ? Severity.WARNING.name() : message.getLevel().toString();
 
-        final Bulletin bulletin = BulletinFactory.createBulletin(null, serviceNode.getIdentifier(), ComponentType.REPORTING_TASK,
-            serviceNode.getName(), "Log Message", bulletinLevel, message.getMessage());
+        final Bulletin bulletin = BulletinFactory.createBulletin(null, serviceNode.getIdentifier(), ComponentType.CONTROLLER_SERVICE,
+                serviceNode.getName(), "Log Message", bulletinLevel, message.getMessage());
         bulletinRepository.addBulletin(bulletin);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/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 903b872..4e50ff3 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
@@ -159,6 +159,7 @@ 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.reporting.ComponentType;
 import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
 import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
@@ -1578,12 +1579,15 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
 
             // get the bulletins for the controller
             final BulletinRepository bulletinRepository = clusterManager.getBulletinRepository();
-            final List<Bulletin> results = bulletinRepository.findBulletinsForController();
-            final List<BulletinDTO> bulletinDtos = new ArrayList<>(results.size());
-            for (final Bulletin bulletin : results) {
-                bulletinDtos.add(dtoFactory.createBulletinDto(bulletin));
-            }
-            controllerStatus.setBulletins(bulletinDtos);
+            controllerStatus.setBulletins(dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForController()));
+
+            // get the controller service bulletins
+            final BulletinQuery controllerServiceQuery = new BulletinQuery.Builder().sourceType(ComponentType.CONTROLLER_SERVICE).build();
+            controllerStatus.setControllerServiceBulletins(dtoFactory.createBulletinDtos(bulletinRepository.findBulletins(controllerServiceQuery)));
+
+            // get the reporting task bulletins
+            final BulletinQuery reportingTaskQuery = new BulletinQuery.Builder().sourceType(ComponentType.REPORTING_TASK).build();
+            controllerStatus.setReportingTaskBulletins(dtoFactory.createBulletinDtos(bulletinRepository.findBulletins(reportingTaskQuery)));
 
             // get the component counts by extracting them from the roots' group status
             final ProcessGroupStatus status = clusterManager.getProcessGroupStatus("root");
@@ -3019,7 +3023,8 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
     }
 
     /**
-     * Utility method for extracting component counts from the specified group status.
+     * Utility method for extracting component counts from the specified group
+     * status.
      */
     private ProcessGroupCounts extractProcessGroupCounts(ProcessGroupStatus groupStatus) {
         int running = 0;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/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 54f11fb..76bce6f 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
@@ -1487,6 +1487,20 @@ public final class DtoFactory {
     }
 
     /**
+     * Creates BulletinDTOs for the specified Bulletins.
+     *
+     * @param bulletins bulletin
+     * @return dto
+     */
+    public List<BulletinDTO> createBulletinDtos(final List<Bulletin> bulletins) {
+        final List<BulletinDTO> bulletinDtos = new ArrayList<>(bulletins.size());
+        for (final Bulletin bulletin : bulletins) {
+            bulletinDtos.add(createBulletinDto(bulletin));
+        }
+        return bulletinDtos;
+    }
+
+    /**
      * Creates a BulletinDTO for the specified Bulletin.
      *
      * @param bulletin bulletin
@@ -1507,7 +1521,8 @@ public final class DtoFactory {
     }
 
     /**
-     * Creates a ProvenanceEventNodeDTO for the specified ProvenanceEventLineageNode.
+     * Creates a ProvenanceEventNodeDTO for the specified
+     * ProvenanceEventLineageNode.
      *
      * @param node node
      * @return dto
@@ -2158,8 +2173,9 @@ public final class DtoFactory {
     /**
      *
      * @param original orig
-     * @param deep if <code>true</code>, all Connections, ProcessGroups, Ports, Processors, etc. will be copied. If <code>false</code>, the copy will have links to the same objects referenced by
-     * <code>original</code>.
+     * @param deep if <code>true</code>, all Connections, ProcessGroups, Ports,
+     * Processors, etc. will be copied. If <code>false</code>, the copy will
+     * have links to the same objects referenced by <code>original</code>.
      *
      * @return dto
      */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
index b614f0a..8bf5553 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
@@ -75,7 +75,6 @@ import org.apache.nifi.provenance.search.SearchTerm;
 import org.apache.nifi.provenance.search.SearchTerms;
 import org.apache.nifi.provenance.search.SearchableField;
 import org.apache.nifi.remote.RootGroupPort;
-import org.apache.nifi.reporting.Bulletin;
 import org.apache.nifi.reporting.BulletinRepository;
 import org.apache.nifi.reporting.ReportingTask;
 import org.apache.nifi.scheduling.SchedulingStrategy;
@@ -88,7 +87,6 @@ import org.apache.nifi.util.FormatUtils;
 import org.apache.nifi.util.NiFiProperties;
 import org.apache.nifi.web.NiFiCoreException;
 import org.apache.nifi.web.ResourceNotFoundException;
-import org.apache.nifi.web.api.dto.BulletinDTO;
 import org.apache.nifi.web.api.dto.DocumentedTypeDTO;
 import org.apache.nifi.web.api.dto.DtoFactory;
 import org.apache.nifi.web.api.dto.provenance.AttributeDTO;
@@ -113,6 +111,8 @@ import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.admin.service.UserService;
 import org.apache.nifi.authorization.DownloadAuthorization;
 import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.reporting.BulletinQuery;
+import org.apache.nifi.reporting.ComponentType;
 import org.apache.nifi.web.security.user.NiFiUserUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -441,12 +441,15 @@ public class ControllerFacade {
         controllerStatus.setQueued(FormatUtils.formatCount(controllerQueueSize.getObjectCount()) + " / " + FormatUtils.formatDataSize(controllerQueueSize.getByteCount()));
 
         final BulletinRepository bulletinRepository = getBulletinRepository();
-        final List<Bulletin> results = bulletinRepository.findBulletinsForController();
-        final List<BulletinDTO> bulletinDtos = new ArrayList<>(results.size());
-        for (final Bulletin bulletin : results) {
-            bulletinDtos.add(dtoFactory.createBulletinDto(bulletin));
-        }
-        controllerStatus.setBulletins(bulletinDtos);
+        controllerStatus.setBulletins(dtoFactory.createBulletinDtos(bulletinRepository.findBulletinsForController()));
+
+        // get the controller service bulletins
+        final BulletinQuery controllerServiceQuery = new BulletinQuery.Builder().sourceType(ComponentType.CONTROLLER_SERVICE).build();
+        controllerStatus.setControllerServiceBulletins(dtoFactory.createBulletinDtos(bulletinRepository.findBulletins(controllerServiceQuery)));
+
+        // get the reporting task bulletins
+        final BulletinQuery reportingTaskQuery = new BulletinQuery.Builder().sourceType(ComponentType.REPORTING_TASK).build();
+        controllerStatus.setReportingTaskBulletins(dtoFactory.createBulletinDtos(bulletinRepository.findBulletins(reportingTaskQuery)));
 
         final ProcessGroupCounts counts = rootGroup.getCounts();
         controllerStatus.setRunningCount(counts.getRunningCount());
@@ -494,36 +497,44 @@ public class ControllerFacade {
     }
 
     /**
-     * Returns the socket port that the Cluster Manager is listening on for Site-to-Site communications
+     * Returns the socket port that the Cluster Manager is listening on for
+     * Site-to-Site communications
      *
-     * @return the socket port that the Cluster Manager is listening on for Site-to-Site communications
+     * @return the socket port that the Cluster Manager is listening on for
+     * Site-to-Site communications
      */
     public Integer getClusterManagerRemoteSiteListeningPort() {
         return flowController.getClusterManagerRemoteSiteListeningPort();
     }
 
     /**
-     * Indicates whether or not Site-to-Site communications with the Cluster Manager are secure
+     * Indicates whether or not Site-to-Site communications with the Cluster
+     * Manager are secure
      *
-     * @return whether or not Site-to-Site communications with the Cluster Manager are secure
+     * @return whether or not Site-to-Site communications with the Cluster
+     * Manager are secure
      */
     public Boolean isClusterManagerRemoteSiteCommsSecure() {
         return flowController.isClusterManagerRemoteSiteCommsSecure();
     }
 
     /**
-     * Returns the socket port that the local instance is listening on for Site-to-Site communications
+     * Returns the socket port that the local instance is listening on for
+     * Site-to-Site communications
      *
-     * @return the socket port that the local instance is listening on for Site-to-Site communications
+     * @return the socket port that the local instance is listening on for
+     * Site-to-Site communications
      */
     public Integer getRemoteSiteListeningPort() {
         return flowController.getRemoteSiteListeningPort();
     }
 
     /**
-     * Indicates whether or not Site-to-Site communications with the local instance are secure
+     * Indicates whether or not Site-to-Site communications with the local
+     * instance are secure
      *
-     * @return whether or not Site-to-Site communications with the local instance are secure
+     * @return whether or not Site-to-Site communications with the local
+     * instance are secure
      */
     public Boolean isRemoteSiteCommsSecure() {
         return flowController.isRemoteSiteCommsSecure();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
index 4a82810..9d87994 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/controller-service.css
@@ -133,9 +133,12 @@ div.referencing-component-bulletins {
     display: none;
 }
 
-span.service.expansion-button {
+div.service.expansion-button {
+    float: left;
     margin-right: 4px;
     margin-top: 2px;
+    width: 10px;
+    height: 10px;
 }
 
 span.referencing-component-active-thread-count {
@@ -197,6 +200,8 @@ div.referencing-component-references {
 
 #disable-controller-service-name {
     float: left;
+    max-width: 280px;
+    text-overflow: ellipsis;
 }
 
 #disable-controller-service-bulletins {
@@ -246,6 +251,8 @@ div.referencing-component-references {
 
 #enable-controller-service-name {
     float: left;
+    max-width: 280px;
+    text-overflow: ellipsis;
 }
 
 #enable-controller-service-bulletins {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.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-canvas.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
index ca45a3d..06ef89f 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas.js
@@ -833,6 +833,9 @@ nf.Canvas = (function () {
                         bulletinIcon.show();
                     }
                 }
+                
+                // update controller service and reporting task bulletins
+                nf.Settings.setBulletins(controllerStatus.controllerServiceBulletins, controllerStatus.reportingTaskBulletins);
 
                 // handle any pending user request
                 if (controllerStatus.hasPendingAccounts === true) {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/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 baa9edc..efcf9fb 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
@@ -149,7 +149,10 @@ nf.ControllerService = (function () {
         // get the table and update the row accordingly
         var controllerServiceGrid = $('#controller-services-table').data('gridInstance');
         var controllerServiceData = controllerServiceGrid.getData();
-        controllerServiceData.updateItem(controllerService.id, controllerService);
+        var currentControllerService = controllerServiceData.getItemById(controllerService.id);
+        controllerServiceData.updateItem(controllerService.id, $.extend({
+            bulletins: currentControllerService.bulletins
+        }, controllerService));
     };
     
     /**
@@ -426,7 +429,7 @@ nf.ControllerService = (function () {
                 
                 // container for this service's references
                 var referencingServiceReferencesContainer = $('<div class="referencing-component-references hidden"></div>');
-                var serviceTwist = $('<span class="service expansion-button collapsed pointer"></span>').on('click', function() {
+                var serviceTwist = $('<div class="service expansion-button collapsed pointer"></div>').on('click', function() {
                     if (serviceTwist.hasClass('collapsed')) {
                         var controllerServiceGrid = $('#controller-services-table').data('gridInstance');
                         var controllerServiceData = controllerServiceGrid.getData();
@@ -511,7 +514,7 @@ nf.ControllerService = (function () {
                 return;
             }
             
-            var twist = $('<span class="expansion-button expanded"></span>');
+            var twist = $('<div class="expansion-button expanded"></div>');
             var title = $('<span class="referencing-component-title"></span>').text(titleText);
             var count = $('<span class="referencing-component-count"></span>').text('(' + list.children().length + ')');
             

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/59aa8ffe/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.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-settings.js b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js
index d323892..3715110 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-settings.js
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-/* global nf, Slick */
+/* global nf, Slick, d3 */
 
 nf.Settings = (function () {
 
@@ -600,9 +600,21 @@ nf.Settings = (function () {
         // more details formatter
         var moreControllerServiceDetails = function (row, cell, value, columnDef, dataContext) {
             var markup = '<img src="images/iconDetails.png" title="View Details" class="pointer view-controller-service" style="margin-top: 5px; float: left;" />';
-            if (!nf.Common.isEmpty(dataContext.validationErrors)) {
-                markup += '<img src="images/iconAlert.png" class="has-errors" style="margin-top: 4px; margin-left: 4px; float: left;" /><span class="hidden row-id">' + nf.Common.escapeHtml(dataContext.id) + '</span>';
+            var hasErrors = !nf.Common.isEmpty(dataContext.validationErrors);
+            var hasBulletins = !nf.Common.isEmpty(dataContext.bulletins);
+            
+            if (hasErrors) {
+                markup += '<img src="images/iconAlert.png" class="has-errors" style="margin-top: 4px; margin-left: 1px; float: left;" />';
+            }
+            
+            if (hasBulletins) {
+                markup += '<img src="images/iconBulletin.png" class="has-bulletins" style="margin-top: 5px; margin-left: 5px; float: left;"/>';
+            }
+            
+            if (hasErrors || hasBulletins) {
+                markup += '<span class="hidden row-id">' + nf.Common.escapeHtml(dataContext.id) + '</span>';
             }
+            
             return markup;
         };
         
@@ -642,7 +654,12 @@ nf.Settings = (function () {
                     markup += '<img src="images/iconDisable.png" title="Disable" class="pointer disable-controller-service" style="margin-top: 2px;" />&nbsp;';
                 } else if (dataContext.state === 'DISABLED') {
                     markup += '<img src="images/iconEdit.png" title="Edit" class="pointer edit-controller-service" style="margin-top: 2px;" />&nbsp;';
-                    markup += '<img src="images/iconEnable.png" title="Enable" class="pointer enable-controller-service" style="margin-top: 2px;"/>&nbsp;';
+                    
+                    // if there are no validation errors allow enabling
+                    if (nf.Common.isEmpty(dataContext.validationErrors)) {
+                        markup += '<img src="images/iconEnable.png" title="Enable" class="pointer enable-controller-service" style="margin-top: 2px;"/>&nbsp;';
+                    }
+                    
                     markup += '<img src="images/iconDelete.png" title="Remove" class="pointer delete-controller-service" style="margin-top: 2px;" />&nbsp;';
                 }
             }
@@ -655,7 +672,7 @@ nf.Settings = (function () {
 
         // define the column model for the controller services table
         var controllerServicesColumns = [
-            {id: 'moreDetails', name: '&nbsp;', resizable: false, formatter: moreControllerServiceDetails, sortable: false, width: 50, maxWidth: 50},
+            {id: 'moreDetails', name: '&nbsp;', resizable: false, formatter: moreControllerServiceDetails, sortable: false, width: 65, maxWidth: 65},
             {id: 'name', field: 'name', name: 'Name', sortable: true, resizable: true},
             {id: 'type', field: 'type', name: 'Type', formatter: typeFormatter, sortable: true, resizable: true},
             {id: 'state', field: 'state', name: 'State', formatter: controllerServiceStateFormatter, sortable: true, resizeable: true}
@@ -767,6 +784,34 @@ nf.Settings = (function () {
                     }, nf.Common.config.tooltipConfig));
                 }
             }
+            
+            var bulletinIcon = $(this).find('img.has-bulletins');
+            if (bulletinIcon.length && !bulletinIcon.data('qtip')) {
+                var taskId = $(this).find('span.row-id').text();
+
+                // get the task item
+                var item = controllerServicesData.getItemById(taskId);
+                
+                // format the tooltip
+                var bulletins = nf.Common.getFormattedBulletins(item.bulletins);
+                var tooltip = nf.Common.formatUnorderedList(bulletins);
+
+                // show the tooltip
+                if (nf.Common.isDefinedAndNotNull(tooltip)) {
+                    bulletinIcon.qtip($.extend({}, nf.Common.config.tooltipConfig, {
+                        content: tooltip,
+                        position: {
+                            target: 'mouse',
+                            viewport: $(window),
+                            adjust: {
+                                x: 8,
+                                y: 8,
+                                method: 'flipinvert flipinvert'
+                            }
+                        }
+                    }));
+                }
+            }
         });
     };
 
@@ -785,7 +830,9 @@ nf.Settings = (function () {
             var nodeServices = response.controllerServices;
             if (nf.Common.isDefinedAndNotNull(nodeServices)) {
                 $.each(nodeServices, function (_, nodeService) {
-                    services.push(nodeService);
+                    services.push($.extend({
+                        bulletins: []
+                    }, nodeService));
                 });
             }
         });
@@ -801,7 +848,9 @@ nf.Settings = (function () {
                     var ncmServices = response.controllerServices;
                     if (nf.Common.isDefinedAndNotNull(ncmServices)) {
                         $.each(ncmServices, function (_, ncmService) {
-                            services.push(ncmService);
+                            services.push($.extend({
+                                bulletins: []
+                            }, ncmService));
                         });
                     }
                     deferred.resolve();
@@ -817,6 +866,7 @@ nf.Settings = (function () {
         return $.when(nodeControllerServices, ncmControllerServices).done(function () {
             var controllerServicesElement = $('#controller-services-table');
             nf.Common.cleanUpTooltips(controllerServicesElement, 'img.has-errors');
+            nf.Common.cleanUpTooltips(controllerServicesElement, 'img.has-bulletins');
 
             var controllerServicesGrid = controllerServicesElement.data('gridInstance');
             var controllerServicesData = controllerServicesGrid.getData();
@@ -1198,9 +1248,21 @@ nf.Settings = (function () {
 
         var moreReportingTaskDetails = function (row, cell, value, columnDef, dataContext) {
             var markup = '<img src="images/iconDetails.png" title="View Details" class="pointer view-reporting-task" style="margin-top: 5px; float: left;" />';
-            if (!nf.Common.isEmpty(dataContext.validationErrors)) {
-                markup += '<img src="images/iconAlert.png" class="has-errors" style="margin-top: 4px; margin-left: 4px; float: left" /><span class="hidden row-id">' + nf.Common.escapeHtml(dataContext.id) + '</span>';
+            var hasErrors = !nf.Common.isEmpty(dataContext.validationErrors);
+            var hasBulletins = !nf.Common.isEmpty(dataContext.bulletins);
+            
+            if (hasErrors) {
+                markup += '<img src="images/iconAlert.png" class="has-errors" style="margin-top: 4px; margin-left: 1px; float: left;" />';
             }
+            
+            if (hasBulletins) {
+                markup += '<img src="images/iconBulletin.png" class="has-bulletins" style="margin-top: 5px; margin-left: 5px; float: left;"/>';
+            }
+            
+            if (hasErrors || hasBulletins) {
+                markup += '<span class="hidden row-id">' + nf.Common.escapeHtml(dataContext.id) + '</span>';
+            }
+            
             return markup;
         };
         
@@ -1239,7 +1301,12 @@ nf.Settings = (function () {
                     markup += '<img src="images/iconStop.png" title="Stop" class="pointer stop-reporting-task" style="margin-top: 2px;" />&nbsp;';
                 } else if (dataContext.state === 'STOPPED' || dataContext.state === 'DISABLED') {
                     markup += '<img src="images/iconEdit.png" title="Edit" class="pointer edit-reporting-task" style="margin-top: 2px;" />&nbsp;';
-                    markup += '<img src="images/iconRun.png" title="Start" class="pointer start-reporting-task" style="margin-top: 2px;"/>&nbsp;';
+                 
+                    // support starting when stopped and no validation errors
+                    if (dataContext.state === 'STOPPED' && nf.Common.isEmpty(dataContext.validationErrors)) {
+                        markup += '<img src="images/iconRun.png" title="Start" class="pointer start-reporting-task" style="margin-top: 2px;"/>&nbsp;';
+                    }
+                    
                     markup += '<img src="images/iconDelete.png" title="Remove" class="pointer delete-reporting-task" style="margin-top: 2px;" />&nbsp;';
                 }
             }
@@ -1252,7 +1319,7 @@ nf.Settings = (function () {
 
         // define the column model for the reporting tasks table
         var reportingTasksColumnModel = [
-            {id: 'moreDetails', field: 'moreDetails', name: '&nbsp;', resizable: false, formatter: moreReportingTaskDetails, sortable: true, width: 50, maxWidth: 50},
+            {id: 'moreDetails', field: 'moreDetails', name: '&nbsp;', resizable: false, formatter: moreReportingTaskDetails, sortable: true, width: 65, maxWidth: 65},
             {id: 'name', field: 'name', name: 'Name', sortable: true, resizable: true},
             {id: 'type', field: 'type', name: 'Type', sortable: true, resizable: true, formatter: typeFormatter},
             {id: 'state', field: 'state', name: 'Run Status', sortable: true, resizeable: true, formatter: reportingTaskRunStatusFormatter}
@@ -1342,7 +1409,7 @@ nf.Settings = (function () {
             if (errorIcon.length && !errorIcon.data('qtip')) {
                 var taskId = $(this).find('span.row-id').text();
 
-                // get the service item
+                // get the task item
                 var item = reportingTasksData.getItemById(taskId);
 
                 // format the errors
@@ -1364,6 +1431,34 @@ nf.Settings = (function () {
                     }, nf.Common.config.tooltipConfig));
                 }
             }
+            
+            var bulletinIcon = $(this).find('img.has-bulletins');
+            if (bulletinIcon.length && !bulletinIcon.data('qtip')) {
+                var taskId = $(this).find('span.row-id').text();
+
+                // get the task item
+                var item = reportingTasksData.getItemById(taskId);
+                
+                // format the tooltip
+                var bulletins = nf.Common.getFormattedBulletins(item.bulletins);
+                var tooltip = nf.Common.formatUnorderedList(bulletins);
+
+                // show the tooltip
+                if (nf.Common.isDefinedAndNotNull(tooltip)) {
+                    bulletinIcon.qtip($.extend({}, nf.Common.config.tooltipConfig, {
+                        content: tooltip,
+                        position: {
+                            target: 'mouse',
+                            viewport: $(window),
+                            adjust: {
+                                x: 8,
+                                y: 8,
+                                method: 'flipinvert flipinvert'
+                            }
+                        }
+                    }));
+                }
+            }
         });
     };
 
@@ -1382,7 +1477,9 @@ nf.Settings = (function () {
             var nodeTasks = response.reportingTasks;
             if (nf.Common.isDefinedAndNotNull(nodeTasks)) {
                 $.each(nodeTasks, function (_, nodeTask) {
-                    tasks.push(nodeTask);
+                    tasks.push($.extend({
+                        bulletins: []
+                    }, nodeTask));
                 });
             }
         });
@@ -1414,6 +1511,7 @@ nf.Settings = (function () {
         return $.when(nodeReportingTasks, ncmReportingTasks).done(function () {
             var reportingTasksElement = $('#reporting-tasks-table');
             nf.Common.cleanUpTooltips(reportingTasksElement, 'img.has-errors');
+            nf.Common.cleanUpTooltips(reportingTasksElement, 'img.has-bulletins');
 
             var reportingTasksGrid = reportingTasksElement.data('gridInstance');
             var reportingTasksData = reportingTasksGrid.getData();
@@ -1580,7 +1678,75 @@ nf.Settings = (function () {
             var reportingTasks = loadReportingTasks();
 
             // return a deferred for all parts of the settings
-            return $.when(settings, controllerServices, reportingTasks).fail(nf.Common.handleAjaxError);
+            return $.when(settings, controllerServices, reportingTasks).done(nf.Canvas.reloadStatus).fail(nf.Common.handleAjaxError);
+        },
+        
+        /**
+         * Sets the controller service and reporting task bulletins in their respective tables.
+         * 
+         * @param {object} controllerServiceBulletins
+         * @param {object} reportingTaskBulletins
+         */
+        setBulletins: function(controllerServiceBulletins, reportingTaskBulletins) {
+            // controller services
+            var controllerServicesGrid = $('#controller-services-table').data('gridInstance');
+            var controllerServicesData = controllerServicesGrid.getData();
+            controllerServicesData.beginUpdate();
+
+            // if there are some bulletins process them
+            if (!nf.Common.isEmpty(controllerServiceBulletins)) {
+                var controllerServiceBulletinsBySource = d3.nest()
+                    .key(function(d) { return d.sourceId; })
+                    .map(controllerServiceBulletins, d3.map);
+            
+                controllerServiceBulletinsBySource.forEach(function(sourceId, sourceBulletins) {
+                    var controllerService = controllerServicesData.getItemById(sourceId);
+                    if (nf.Common.isDefinedAndNotNull(controllerService)) {
+                        controllerServicesData.updateItem(sourceId, $.extend(controllerService, {
+                            bulletins: sourceBulletins
+                        }));
+                    }
+                });
+            } else {
+                // if there are no bulletins clear all
+                var controllerServices = controllerServicesData.getItems();
+                $.each(controllerServices, function(_, controllerService) {
+                    controllerServicesData.updateItem(controllerService.id, $.extend(controllerService, {
+                        bulletins: []
+                    }));
+                });
+            }
+            controllerServicesData.endUpdate();
+
+            // reporting tasks
+            var reportingTasksGrid = $('#reporting-tasks-table').data('gridInstance');
+            var reportingTasksData = reportingTasksGrid.getData();
+            reportingTasksData.beginUpdate();
+            
+            // if there are some bulletins process them
+            if (!nf.Common.isEmpty(reportingTaskBulletins)) {
+                var reportingTaskBulletinsBySource = d3.nest()
+                    .key(function(d) { return d.sourceId; })
+                    .map(reportingTaskBulletins, d3.map);
+
+                reportingTaskBulletinsBySource.forEach(function(sourceId, sourceBulletins) {
+                    var reportingTask = reportingTasksData.getItemById(sourceId);
+                    if (nf.Common.isDefinedAndNotNull(reportingTask)) {
+                        reportingTasksData.updateItem(sourceId, $.extend(reportingTask, {
+                            bulletins: sourceBulletins
+                        }));
+                    }
+                });
+            } else {
+                // if there are no bulletins clear all
+                var reportingTasks = reportingTasksData.getItems();
+                $.each(reportingTasks, function(_, reportingTask) {
+                    controllerServicesData.updateItem(reportingTask.id, $.extend(reportingTask, {
+                        bulletins: []
+                    }));
+                });
+            }
+            reportingTasksData.endUpdate();
         }
     };
 }());
\ No newline at end of file