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/04/23 19:54:10 UTC

[2/2] incubator-nifi git commit: NIFI-524: - Adding a Process Group tab to the summary table to show aggregate statistics of their comonents.

NIFI-524:
- Adding a Process Group tab to the summary table to show aggregate statistics of their comonents.

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

Branch: refs/heads/develop
Commit: ff831dc33de98a01f53c79d5c6f04bf93bbaef69
Parents: 8bfc969
Author: Matt Gilman <ma...@gmail.com>
Authored: Thu Apr 23 12:59:41 2015 -0400
Committer: Matt Gilman <ma...@gmail.com>
Committed: Thu Apr 23 12:59:41 2015 -0400

----------------------------------------------------------------------
 .../controller/status/ProcessGroupStatus.java   |  38 ++
 .../status/ClusterProcessGroupStatusDTO.java    |  89 ++++
 .../dto/status/NodeProcessGroupStatusDTO.java   |  57 +++
 .../api/dto/status/ProcessGroupStatusDTO.java   |  73 +++
 .../entity/ClusterProcessGroupStatusEntity.java |  45 ++
 .../apache/nifi/controller/FlowController.java  |  10 +
 .../org/apache/nifi/web/NiFiServiceFacade.java  |  57 ++-
 .../nifi/web/StandardNiFiServiceFacade.java     |  73 +++
 .../apache/nifi/web/api/ClusterResource.java    |  94 ++--
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  18 +-
 .../src/main/webapp/WEB-INF/pages/summary.jsp   |   1 +
 .../cluster-process-group-summary-dialog.jsp    |  36 ++
 .../partials/summary/summary-content.jsp        |   3 +
 .../nifi-web-ui/src/main/webapp/css/summary.css |  83 ++++
 .../src/main/webapp/images/iconProcessGroup.png | Bin 0 -> 1314 bytes
 .../webapp/js/nf/summary/nf-summary-table.js    | 448 +++++++++++++++++--
 16 files changed, 1002 insertions(+), 123 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
index 7aae866..45acf8e 100644
--- a/nifi/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
+++ b/nifi/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
@@ -43,6 +43,8 @@ public class ProcessGroupStatus implements Cloneable {
     private long bytesReceived;
     private int flowFilesSent;
     private long bytesSent;
+    private int flowFilesTransferred;
+    private long bytesTransferred;
 
     private Collection<ConnectionStatus> connectionStatus = new ArrayList<>();
     private Collection<ProcessorStatus> processorStatus = new ArrayList<>();
@@ -227,6 +229,22 @@ public class ProcessGroupStatus implements Cloneable {
         this.bytesSent = bytesSent;
     }
 
+    public int getFlowFilesTransferred() {
+        return flowFilesTransferred;
+    }
+
+    public void setFlowFilesTransferred(int flowFilesTransferred) {
+        this.flowFilesTransferred = flowFilesTransferred;
+    }
+
+    public long getBytesTransferred() {
+        return bytesTransferred;
+    }
+
+    public void setBytesTransferred(long bytesTransferred) {
+        this.bytesTransferred = bytesTransferred;
+    }
+
     @Override
     public ProcessGroupStatus clone() {
 
@@ -248,6 +266,8 @@ public class ProcessGroupStatus implements Cloneable {
         clonedObj.bytesReceived = bytesReceived;
         clonedObj.flowFilesSent = flowFilesSent;
         clonedObj.bytesSent = bytesSent;
+        clonedObj.flowFilesTransferred = flowFilesTransferred;
+        clonedObj.bytesTransferred = bytesTransferred;
 
         if (connectionStatus != null) {
             final Collection<ConnectionStatus> statusList = new ArrayList<>();
@@ -317,6 +337,18 @@ public class ProcessGroupStatus implements Cloneable {
         builder.append(creationTimestamp);
         builder.append(", activeThreadCount=");
         builder.append(activeThreadCount);
+        builder.append(", flowFilesTransferred=");
+        builder.append(flowFilesTransferred);
+        builder.append(", bytesTransferred=");
+        builder.append(bytesTransferred);
+        builder.append(", flowFilesReceived=");
+        builder.append(flowFilesReceived);
+        builder.append(", bytesReceived=");
+        builder.append(bytesReceived);
+        builder.append(", flowFilesSent=");
+        builder.append(flowFilesSent);
+        builder.append(", bytesSent=");
+        builder.append(bytesSent);
         builder.append(",\n\tconnectionStatus=");
 
         for (final ConnectionStatus status : connectionStatus) {
@@ -374,6 +406,12 @@ public class ProcessGroupStatus implements Cloneable {
         target.setBytesRead(target.getBytesRead() + toMerge.getBytesRead());
         target.setBytesWritten(target.getBytesWritten() + toMerge.getBytesWritten());
         target.setActiveThreadCount(target.getActiveThreadCount() + toMerge.getActiveThreadCount());
+        target.setFlowFilesTransferred(target.getFlowFilesTransferred() + toMerge.getFlowFilesTransferred());
+        target.setBytesTransferred(target.getBytesTransferred() + toMerge.getBytesTransferred());
+        target.setFlowFilesReceived(target.getFlowFilesReceived() + toMerge.getFlowFilesReceived());
+        target.setBytesReceived(target.getBytesReceived() + toMerge.getBytesReceived());
+        target.setFlowFilesSent(target.getFlowFilesSent() + toMerge.getFlowFilesSent());
+        target.setBytesSent(target.getBytesSent() + toMerge.getBytesSent());
 
         // connection status
         // sort by id

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessGroupStatusDTO.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/ClusterProcessGroupStatusDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessGroupStatusDTO.java
new file mode 100644
index 0000000..08d76a5
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessGroupStatusDTO.java
@@ -0,0 +1,89 @@
+/*
+ * 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.web.api.dto.status;
+
+import java.util.Collection;
+import java.util.Date;
+import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
+
+/**
+ * DTO for serializing the a process group's status across the cluster.
+ */
+@XmlType(name = "clusterProcessGroupStatus")
+public class ClusterProcessGroupStatusDTO {
+
+    private Collection<NodeProcessGroupStatusDTO> nodeProcessGroupStatus;
+    private Date statsLastRefreshed;
+    private String processGroupId;
+    private String processGroupName;
+
+    /**
+     * The time the status were last refreshed.
+     *
+     * @return The time the status were last refreshed
+     */
+    @XmlJavaTypeAdapter(TimeAdapter.class)
+    public Date getStatsLastRefreshed() {
+        return statsLastRefreshed;
+    }
+
+    public void setStatsLastRefreshed(Date statsLastRefreshed) {
+        this.statsLastRefreshed = statsLastRefreshed;
+    }
+
+    /**
+     * The process group id.
+     *
+     * @return The process group id
+     */
+    public String getProcessGroupId() {
+        return processGroupId;
+    }
+
+    public void setProcessGroupId(String processGroupId) {
+        this.processGroupId = processGroupId;
+    }
+
+    /**
+     * The process group name.
+     *
+     * @return The process group name
+     */
+    public String getProcessGroupName() {
+        return processGroupName;
+    }
+
+    public void setProcessGroupName(String processGroupName) {
+        this.processGroupName = processGroupName;
+    }
+
+    /**
+     * Collection of node process group status DTO.
+     *
+     * @return The collection of node process group status DTO
+     */
+    public Collection<NodeProcessGroupStatusDTO> getNodeProcessGroupStatus() {
+        return nodeProcessGroupStatus;
+    }
+
+    public void setNodeProcessGroupStatus(Collection<NodeProcessGroupStatusDTO> nodeProcessGroupStatus) {
+        this.nodeProcessGroupStatus = nodeProcessGroupStatus;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusDTO.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/NodeProcessGroupStatusDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusDTO.java
new file mode 100644
index 0000000..5f965b2
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusDTO.java
@@ -0,0 +1,57 @@
+/*
+ * 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.web.api.dto.status;
+
+import javax.xml.bind.annotation.XmlType;
+import org.apache.nifi.web.api.dto.NodeDTO;
+
+/**
+ * DTO for serializing the process group status for a particular node.
+ */
+@XmlType(name = "nodeProcessGroupStatus")
+public class NodeProcessGroupStatusDTO {
+
+    private NodeDTO node;
+    private ProcessGroupStatusDTO processGroupStatus;
+
+    /**
+     * The node.
+     *
+     * @return The node DTO
+     */
+    public NodeDTO getNode() {
+        return node;
+    }
+
+    public void setNode(NodeDTO node) {
+        this.node = node;
+    }
+
+    /**
+     * The process group's status.
+     *
+     * @return The process group status
+     */
+    public ProcessGroupStatusDTO getProcessGroupStatus() {
+        return processGroupStatus;
+    }
+
+    public void setProcessGroupStatus(ProcessGroupStatusDTO processGroupStatus) {
+        this.processGroupStatus = processGroupStatus;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.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/ProcessGroupStatusDTO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
index 2193fb0..7ad24a9 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
@@ -38,10 +38,15 @@ public class ProcessGroupStatusDTO extends StatusDTO {
     private Collection<PortStatusDTO> outputPortStatus;
 
     private String input;
+    private String queuedCount;
+    private String queuedSize;
     private String queued;
     private String read;
     private String written;
     private String output;
+    private String transferred;
+    private String received;
+    private String sent;
     private Integer activeThreadCount;
     private Date statsLastRefreshed;
 
@@ -172,6 +177,74 @@ public class ProcessGroupStatusDTO extends StatusDTO {
     }
 
     /**
+     * The transferred stats for this process group. This represents the
+     * count/size of flowfiles transferred to/from queues.
+     *
+     * @return The transferred status for this process group
+     */
+    public String getTransferred() {
+        return transferred;
+    }
+
+    public void setTransferred(String transferred) {
+        this.transferred = transferred;
+    }
+
+    /**
+     * The received stats for this process group. This represents the count/size
+     * of flowfiles received.
+     *
+     * @return The received stats for this process group
+     */
+    public String getReceived() {
+        return received;
+    }
+
+    public void setReceived(String received) {
+        this.received = received;
+    }
+
+    /**
+     * The sent stats for this process group. This represents the count/size of
+     * flowfiles sent.
+     *
+     * @return The sent stats for this process group
+     */
+    public String getSent() {
+        return sent;
+    }
+
+    public void setSent(String sent) {
+        this.sent = sent;
+    }
+
+    /**
+     * The queued count for this process group.
+     *
+     * @return The queued count for this process group
+     */
+    public String getQueuedCount() {
+        return queuedCount;
+    }
+
+    public void setQueuedCount(String queuedCount) {
+        this.queuedCount = queuedCount;
+    }
+
+    /**
+     * The queued size for this process group.
+     *
+     * @return The queued size for this process group
+     */
+    public String getQueuedSize() {
+        return queuedSize;
+    }
+
+    public void setQueuedSize(String queuedSize) {
+        this.queuedSize = queuedSize;
+    }
+
+    /**
      * The queued stats for this process group.
      *
      * @return The queued stats

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessGroupStatusEntity.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessGroupStatusEntity.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessGroupStatusEntity.java
new file mode 100644
index 0000000..cddb21a
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessGroupStatusEntity.java
@@ -0,0 +1,45 @@
+/*
+ * 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.web.api.entity;
+
+import javax.xml.bind.annotation.XmlRootElement;
+import org.apache.nifi.web.api.dto.status.ClusterProcessGroupStatusDTO;
+
+/**
+ * A serialized representation of this class can be placed in the entity body of
+ * a request or response to or from the API. This particular entity holds a
+ * reference to a ClusterProcessGroupStatusDTO.
+ */
+@XmlRootElement(name = "clusterProcessGroupStatusEntity")
+public class ClusterProcessGroupStatusEntity extends Entity {
+
+    private ClusterProcessGroupStatusDTO clusterProcessGroupStatus;
+
+    /**
+     * The ClusterProcessGroupStatusDTO that is being serialized.
+     *
+     * @return The ClusterProcessGroupStatusDTO object
+     */
+    public ClusterProcessGroupStatusDTO getClusterProcessGroupStatus() {
+        return clusterProcessGroupStatus;
+    }
+
+    public void setClusterProcessGroupStatus(ClusterProcessGroupStatusDTO clusterProcessGroupStatus) {
+        this.clusterProcessGroupStatus = clusterProcessGroupStatus;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
index 07e754e..0d7699a 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java
@@ -2063,6 +2063,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         long bytesReceived = 0L;
         int flowFilesSent = 0;
         long bytesSent = 0L;
+        int flowFilesTransferred = 0;
+        long bytesTransferred = 0;
 
         // set status for processors
         final Collection<ProcessorStatus> processorStatusCollection = new ArrayList<>();
@@ -2096,6 +2098,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
             bytesReceived += childGroupStatus.getBytesReceived();
             flowFilesSent += childGroupStatus.getFlowFilesSent();
             bytesSent += childGroupStatus.getBytesSent();
+
+            flowFilesTransferred += childGroupStatus.getFlowFilesTransferred();
+            bytesTransferred += childGroupStatus.getBytesTransferred();
         }
 
         // set status for remote child groups
@@ -2133,6 +2138,9 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
                 connStatus.setInputCount(connectionStatusReport.getFlowFilesIn());
                 connStatus.setOutputBytes(connectionStatusReport.getContentSizeOut());
                 connStatus.setOutputCount(connectionStatusReport.getFlowFilesOut());
+
+                flowFilesTransferred += (connectionStatusReport.getFlowFilesIn() + connectionStatusReport.getFlowFilesOut());
+                bytesTransferred += (connectionStatusReport.getContentSizeIn() + connectionStatusReport.getContentSizeOut());
             }
 
             if (StringUtils.isNotBlank(conn.getName())) {
@@ -2303,6 +2311,8 @@ public class FlowController implements EventAccess, ControllerServiceProvider, R
         status.setBytesReceived(bytesReceived);
         status.setFlowFilesSent(flowFilesSent);
         status.setBytesSent(bytesSent);
+        status.setFlowFilesTransferred(flowFilesTransferred);
+        status.setBytesTransferred(bytesTransferred);
 
         return status;
     }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/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 6cf22c0..c98b1e4 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
@@ -63,6 +63,7 @@ import org.apache.nifi.web.api.dto.provenance.lineage.LineageDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
 import org.apache.nifi.web.api.dto.status.ClusterConnectionStatusDTO;
 import org.apache.nifi.web.api.dto.status.ClusterPortStatusDTO;
+import org.apache.nifi.web.api.dto.status.ClusterProcessGroupStatusDTO;
 import org.apache.nifi.web.api.dto.status.ClusterProcessorStatusDTO;
 import org.apache.nifi.web.api.dto.status.ClusterRemoteProcessGroupStatusDTO;
 import org.apache.nifi.web.api.dto.status.ClusterStatusDTO;
@@ -774,8 +775,7 @@ public interface NiFiServiceFacade {
     void verifyUpdateRemoteProcessGroup(String groupId, RemoteProcessGroupDTO remoteProcessGroupDTO);
 
     /**
-     * Verifies the specified remote process group can update the specified
-     * remote input port.
+     * Verifies the specified remote process group can update the specified remote input port.
      *
      * @param groupId The id of the parent group
      * @param remoteProcessGroupId The id of the remote process group
@@ -784,8 +784,7 @@ public interface NiFiServiceFacade {
     void verifyUpdateRemoteProcessGroupInputPort(String groupId, String remoteProcessGroupId, RemoteProcessGroupPortDTO remoteProcessGroupPortDTO);
 
     /**
-     * Verifies the specified remote process group can update the specified
-     * remote output port.
+     * Verifies the specified remote process group can update the specified remote output port.
      *
      * @param groupId The id of the parent group
      * @param remoteProcessGroupId The id of the remote process group
@@ -977,13 +976,12 @@ public interface NiFiServiceFacade {
      * Gets the specified controller service.
      *
      * @param controllerServiceId id
-     * @return service 
+     * @return service
      */
     ControllerServiceDTO getControllerService(String controllerServiceId);
 
     /**
-     * Get the descriptor for the specified property of the specified controller
-     * service.
+     * Get the descriptor for the specified property of the specified controller service.
      *
      * @param id id
      * @param property property
@@ -1037,8 +1035,7 @@ public interface NiFiServiceFacade {
     void verifyUpdateControllerService(ControllerServiceDTO controllerServiceDTO);
 
     /**
-     * Verifies the referencing components of the specified controller service
-     * can be updated.
+     * Verifies the referencing components of the specified controller service can be updated.
      *
      * @param controllerServiceId id
      * @param scheduledState schedule state
@@ -1081,8 +1078,7 @@ public interface NiFiServiceFacade {
     ReportingTaskDTO getReportingTask(String reportingTaskId);
 
     /**
-     * Get the descriptor for the specified property of the specified reporting
-     * task.
+     * Get the descriptor for the specified property of the specified reporting task.
      *
      * @param id id
      * @param property property
@@ -1196,8 +1192,7 @@ public interface NiFiServiceFacade {
     void verifyUpdateSnippet(SnippetDTO snippetDto);
 
     /**
-     * If group id is specified, moves the specified snippet to the specified
-     * group.
+     * If group id is specified, moves the specified snippet to the specified group.
      *
      * @param revision revision
      * @param snippetDto snippet
@@ -1256,8 +1251,7 @@ public interface NiFiServiceFacade {
     void invalidateUser(String userId);
 
     /**
-     * Invalidates the specified user accounts and all accounts associated with
-     * this group.
+     * Invalidates the specified user accounts and all accounts associated with this group.
      *
      * @param userGroup group
      * @param userIds id
@@ -1272,8 +1266,7 @@ public interface NiFiServiceFacade {
     void deleteUser(String userId);
 
     /**
-     * Updates a user group with the specified group and comprised of the
-     * specified users.
+     * Updates a user group with the specified group and comprised of the specified users.
      *
      * @param userGroup group
      * @return group
@@ -1298,8 +1291,7 @@ public interface NiFiServiceFacade {
     // Cluster methods
     // ----------------------------------------
     /**
-     * @return true if controller is connected or trying to connect to the
-     * cluster
+     * @return true if controller is connected or trying to connect to the cluster
      */
     boolean isClustered();
 
@@ -1369,8 +1361,7 @@ public interface NiFiServiceFacade {
 
     /**
      * @param processorId id
-     * @return the processor status history for each node connected to the
-     * cluster
+     * @return the processor status history for each node connected to the cluster
      */
     ClusterStatusHistoryDTO getClusterProcessorStatusHistory(String processorId);
 
@@ -1384,28 +1375,34 @@ public interface NiFiServiceFacade {
 
     /**
      * @param connectionId id
-     * @return the connection status history for each node connected to the
-     * cluster
+     * @return the connection status history for each node connected to the cluster
      */
     ClusterStatusHistoryDTO getClusterConnectionStatusHistory(String connectionId);
 
     /**
      * @param processGroupId id
-     * @return the process group status history for each node connected to the
-     * cluster
+     * @return the process group status history for each node connected to the cluster
      */
     ClusterStatusHistoryDTO getClusterProcessGroupStatusHistory(String processGroupId);
 
     /**
-     * @param remoteProcessGroupId id
-     * @return the remote process group status history for each node connected
-     * to the cluster
+     * Returns a process group's status for each node connected to the cluster.
+     *
+     * @param processorId a process group identifier
+     * @return The cluster process group status transfer object.
+     */
+    ClusterProcessGroupStatusDTO getClusterProcessGroupStatus(String processorId);
+
+    /**
+     * Returns the remote process group status history for each node connected to the cluster.
+     *
+     * @param remoteProcessGroupId a remote process group identifier
+     * @return The cluster status history
      */
     ClusterStatusHistoryDTO getClusterRemoteProcessGroupStatusHistory(String remoteProcessGroupId);
 
     /**
-     * Returns a remote process group's status for each node connected to the
-     * cluster.
+     * Returns a remote process group's status for each node connected to the cluster.
      *
      * @param remoteProcessGroupId a remote process group identifier
      * @return The cluster remote process group status transfer object.

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/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 df4cdf1..fbd4742 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
@@ -163,6 +163,8 @@ import org.apache.nifi.web.api.dto.ControllerServiceDTO;
 import org.apache.nifi.web.api.dto.ControllerServiceReferencingComponentDTO;
 import org.apache.nifi.web.api.dto.PropertyDescriptorDTO;
 import org.apache.nifi.web.api.dto.ReportingTaskDTO;
+import org.apache.nifi.web.api.dto.status.ClusterProcessGroupStatusDTO;
+import org.apache.nifi.web.api.dto.status.NodeProcessGroupStatusDTO;
 import org.apache.nifi.web.dao.ControllerServiceDAO;
 import org.apache.nifi.web.dao.ReportingTaskDAO;
 import org.slf4j.Logger;
@@ -2447,6 +2449,77 @@ public class StandardNiFiServiceFacade implements NiFiServiceFacade {
         return clusterConnectionStatusDto;
     }
 
+    private ProcessGroupStatus findNodeProcessGroupStatus(final ProcessGroupStatus groupStatus, final String processGroupId) {
+        ProcessGroupStatus processGroupStatus = null;
+
+        if (processGroupId.equals(groupStatus.getId())) {
+            processGroupStatus = groupStatus;
+        }
+
+        if (processGroupStatus == null) {
+            for (final ProcessGroupStatus status : groupStatus.getProcessGroupStatus()) {
+                processGroupStatus = findNodeProcessGroupStatus(status, processGroupId);
+
+                if (processGroupStatus != null) {
+                    break;
+                }
+            }
+        }
+
+        return processGroupStatus;
+    }
+
+    @Override
+    public ClusterProcessGroupStatusDTO getClusterProcessGroupStatus(String processGroupId) {
+
+        final ClusterProcessGroupStatusDTO clusterProcessGroupStatusDto = new ClusterProcessGroupStatusDTO();
+        clusterProcessGroupStatusDto.setNodeProcessGroupStatus(new ArrayList<NodeProcessGroupStatusDTO>());
+
+        // set the current time
+        clusterProcessGroupStatusDto.setStatsLastRefreshed(new Date());
+
+        final Set<Node> nodes = clusterManager.getNodes(Node.Status.CONNECTED);
+        boolean firstNode = true;
+        for (final Node node : nodes) {
+
+            final HeartbeatPayload nodeHeartbeatPayload = node.getHeartbeatPayload();
+            if (nodeHeartbeatPayload == null) {
+                continue;
+            }
+
+            final ProcessGroupStatus nodeStats = nodeHeartbeatPayload.getProcessGroupStatus();
+            if (nodeStats == null || nodeStats.getProcessorStatus() == null) {
+                continue;
+            }
+
+            // attempt to find the process group stats for this node
+            final ProcessGroupStatus processGroupStatus = findNodeProcessGroupStatus(nodeStats, processGroupId);
+
+            // sanity check that we have status for this process group
+            if (processGroupStatus == null) {
+                throw new ResourceNotFoundException(String.format("Unable to find status for process group id '%s'.", processGroupId));
+            }
+
+            if (firstNode) {
+                clusterProcessGroupStatusDto.setProcessGroupId(processGroupId);
+                clusterProcessGroupStatusDto.setProcessGroupName(processGroupStatus.getName());
+                firstNode = false;
+            }
+
+            // create node process group status dto
+            final NodeProcessGroupStatusDTO nodeProcessGroupStatusDTO = new NodeProcessGroupStatusDTO();
+            clusterProcessGroupStatusDto.getNodeProcessGroupStatus().add(nodeProcessGroupStatusDTO);
+
+            // populate node process group status dto
+            final String nodeId = node.getNodeId().getId();
+            nodeProcessGroupStatusDTO.setNode(dtoFactory.createNodeDTO(node, clusterManager.getNodeEvents(nodeId), isPrimaryNode(nodeId)));
+            nodeProcessGroupStatusDTO.setProcessGroupStatus(dtoFactory.createProcessGroupStatusDto(clusterManager.getBulletinRepository(), processGroupStatus));
+
+        }
+
+        return clusterProcessGroupStatusDto;
+    }
+
     private PortStatus findNodeInputPortStatus(final ProcessGroupStatus groupStatus, final String inputPortId) {
         PortStatus portStatus = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.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/ClusterResource.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java
index 3a74782..a99d7df 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/api/ClusterResource.java
@@ -69,6 +69,8 @@ import org.apache.commons.lang3.StringUtils;
 import org.springframework.security.access.prepost.PreAuthorize;
 
 import com.sun.jersey.api.core.ResourceContext;
+import org.apache.nifi.web.api.dto.status.ClusterProcessGroupStatusDTO;
+import org.apache.nifi.web.api.entity.ClusterProcessGroupStatusEntity;
 import org.codehaus.enunciate.jaxrs.TypeHint;
 
 /**
@@ -95,9 +97,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the status of this NiFi cluster.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @return A clusterStatusEntity
      */
     @GET
@@ -144,12 +144,9 @@ public class ClusterResource extends ApplicationResource {
     }
 
     /**
-     * Gets the contents of this NiFi cluster. This includes all nodes and their
-     * status.
+     * Gets the contents of this NiFi cluster. This includes all nodes and their status.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @return A clusterEntity
      */
     @GET
@@ -231,9 +228,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the processor.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the processor
      * @return A processorEntity
      */
@@ -267,11 +262,8 @@ public class ClusterResource extends ApplicationResource {
      * Updates the processors annotation data.
      *
      * @param httpServletRequest
-     * @param version The revision is used to verify the client is working with
-     * the latest version of the flow.
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param version The revision is used to verify the client is working with the latest version of the flow.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param processorId The id of the processor.
      * @param annotationData The annotation data to set.
      * @return A processorEntity.
@@ -395,9 +387,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the processor status for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the processor
      * @return A clusterProcessorStatusEntity
      */
@@ -431,9 +421,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the processor status history for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the processor
      * @return A clusterProcessorStatusHistoryEntity
      */
@@ -466,9 +454,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the connection status for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the processor
      * @return A clusterProcessorStatusEntity
      */
@@ -502,9 +488,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the connections status history for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the processor
      * @return A clusterProcessorStatusHistoryEntity
      */
@@ -535,11 +519,43 @@ public class ClusterResource extends ApplicationResource {
     }
 
     /**
+     * Gets the process group status for every node.
+     *
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
+     * @param id The id of the process group
+     * @return A clusterProcessGroupStatusEntity
+     */
+    @GET
+    @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+    @Path("/process-groups/{id}/status")
+    @PreAuthorize("hasAnyRole('ROLE_MONITOR', 'ROLE_DFM', 'ROLE_ADMIN')")
+    @TypeHint(ClusterConnectionStatusEntity.class)
+    public Response getProcessGroupStatus(@QueryParam(CLIENT_ID) @DefaultValue(StringUtils.EMPTY) ClientIdParameter clientId, @PathParam("id") String id) {
+
+        if (properties.isClusterManager()) {
+
+            final ClusterProcessGroupStatusDTO dto = serviceFacade.getClusterProcessGroupStatus(id);
+
+            // create the revision
+            RevisionDTO revision = new RevisionDTO();
+            revision.setClientId(clientId.getClientId());
+
+            // create entity
+            final ClusterProcessGroupStatusEntity entity = new ClusterProcessGroupStatusEntity();
+            entity.setClusterProcessGroupStatus(dto);
+            entity.setRevision(revision);
+
+            // generate the response
+            return generateOkResponse(entity).build();
+        }
+
+        throw new IllegalClusterResourceRequestException("Only a cluster manager can process the request.");
+    }
+
+    /**
      * Gets the process group status history for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the process group
      * @return A clusterProcessGroupStatusHistoryEntity
      */
@@ -572,9 +588,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the remote process group status for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the remote process group
      * @return A clusterRemoteProcessGroupStatusEntity
      */
@@ -608,9 +622,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the input port status for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the input port
      * @return A clusterPortStatusEntity
      */
@@ -644,9 +656,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the output port status for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the output port
      * @return A clusterPortStatusEntity
      */
@@ -680,9 +690,7 @@ public class ClusterResource extends ApplicationResource {
     /**
      * Gets the remote process group status history for every node.
      *
-     * @param clientId Optional client id. If the client id is not specified, a
-     * new one will be generated. This value (whether specified or generated) is
-     * included in the response.
+     * @param clientId Optional client id. If the client id is not specified, a new one will be generated. This value (whether specified or generated) is included in the response.
      * @param id The id of the processor
      * @return A clusterRemoteProcessGroupStatusHistoryEntity
      */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/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 56ee9ba..2402b73 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
@@ -506,13 +506,21 @@ public final class DtoFactory {
         processGroupStatusDto.setId(processGroupStatus.getId());
         processGroupStatusDto.setName(processGroupStatus.getName());
         processGroupStatusDto.setStatsLastRefreshed(new Date(processGroupStatus.getCreationTimestamp()));
-        processGroupStatusDto.setQueued(formatCount(processGroupStatus.getQueuedCount()) + " / " + formatDataSize(processGroupStatus.getQueuedContentSize()));
         processGroupStatusDto.setRead(formatDataSize(processGroupStatus.getBytesRead()));
         processGroupStatusDto.setWritten(formatDataSize(processGroupStatus.getBytesWritten()));
         processGroupStatusDto.setInput(formatCount(processGroupStatus.getInputCount()) + " / " + formatDataSize(processGroupStatus.getInputContentSize()));
         processGroupStatusDto.setOutput(formatCount(processGroupStatus.getOutputCount()) + " / " + formatDataSize(processGroupStatus.getOutputContentSize()));
+        processGroupStatusDto.setTransferred(formatCount(processGroupStatus.getFlowFilesTransferred()) + " / " + formatDataSize(processGroupStatus.getBytesTransferred()));
+        processGroupStatusDto.setSent(formatCount(processGroupStatus.getFlowFilesSent()) + " / " + formatDataSize(processGroupStatus.getBytesSent()));
+        processGroupStatusDto.setReceived(formatCount(processGroupStatus.getFlowFilesReceived()) + " / " + formatDataSize(processGroupStatus.getBytesReceived()));
         processGroupStatusDto.setActiveThreadCount(processGroupStatus.getActiveThreadCount());
 
+        final String queuedCount = FormatUtils.formatCount(processGroupStatus.getQueuedCount());
+        final String queuedSize = FormatUtils.formatDataSize(processGroupStatus.getQueuedContentSize());
+        processGroupStatusDto.setQueuedCount(queuedCount);
+        processGroupStatusDto.setQueuedSize(queuedSize);
+        processGroupStatusDto.setQueued(queuedCount + " / " + queuedSize);
+
         final Map<String, StatusDTO> componentStatusDtoMap = new HashMap<>();
 
         // processor status
@@ -1504,8 +1512,7 @@ public final class DtoFactory {
     }
 
     /**
-     * Creates a ProvenanceEventNodeDTO for the specified
-     * ProvenanceEventLineageNode.
+     * Creates a ProvenanceEventNodeDTO for the specified ProvenanceEventLineageNode.
      *
      * @param node
      * @return
@@ -2158,9 +2165,8 @@ public final class DtoFactory {
     /**
      *
      * @param original
-     * @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
      */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp
index 032509b..e6f3305 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/pages/summary.jsp
@@ -74,6 +74,7 @@
         <jsp:include page="/WEB-INF/partials/summary/cluster-output-port-summary-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/summary/cluster-remote-process-group-summary-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/summary/cluster-connection-summary-dialog.jsp"/>
+        <jsp:include page="/WEB-INF/partials/summary/cluster-process-group-summary-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/summary/system-diagnostics-dialog.jsp"/>
         <jsp:include page="/WEB-INF/partials/summary/view-single-node-dialog.jsp"/>
         <div id="faded-background"></div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/cluster-process-group-summary-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/summary/cluster-process-group-summary-dialog.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/cluster-process-group-summary-dialog.jsp
new file mode 100644
index 0000000..94526d0
--- /dev/null
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/cluster-process-group-summary-dialog.jsp
@@ -0,0 +1,36 @@
+<%--
+ 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="cluster-process-group-summary-dialog">
+    <div class="dialog-content">
+        <div id="cluster-process-group-summary-header">
+            <div id="cluster-process-group-refresh-button" class="summary-refresh pointer" title="Refresh"></div>
+            <div id="cluster-process-group-summary-last-refreshed-container">
+                Last updated:&nbsp;<span id="cluster-process-group-summary-last-refreshed"></span>
+            </div>
+            <div id="cluster-process-group-summary-loading-container" class="loading-container"></div>
+            <div id="cluster-process-group-details-container">
+                <div id="cluster-process-group-icon"></div>
+                <div id="cluster-process-group-details">
+                    <div id="cluster-process-group-name"></div>
+                    <div id="cluster-process-group-id"></div>
+                </div>
+            </div>
+        </div>
+        <div id="cluster-process-group-summary-table"></div>
+    </div>
+</div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/summary-content.jsp
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/summary-content.jsp b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/summary-content.jsp
index a419baa..5be3e2b 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/summary-content.jsp
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/WEB-INF/partials/summary/summary-content.jsp
@@ -51,6 +51,9 @@
         <div id="connection-summary-tab-content" class="configuration-tab">
             <div id="connection-summary-table" class="summary-table"></div>
         </div>
+        <div id="process-group-summary-tab-content" class="configuration-tab">
+            <div id="process-group-summary-table" class="summary-table"></div>
+        </div>
         <div id="input-port-summary-tab-content" class="configuration-tab">
             <div id="input-port-summary-table" class="summary-table"></div>
         </div>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/summary.css
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/summary.css b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/summary.css
index bb7b9c4..e882f89 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/summary.css
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/css/summary.css
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 /*
     Status Styles
 */
@@ -386,6 +387,12 @@ span.sorted {
     text-decoration: underline;
 }
 
+/* tooltips in the summary table */
+
+#summary .nifi-tooltip {
+    max-width: 500px;
+}
+
 /* cluster processor summary table */
 
 #cluster-processor-summary-dialog {
@@ -765,4 +772,80 @@ span.sorted {
     white-space: nowrap;
     overflow: hidden;
     width: 200px;
+}
+
+/* cluster process group summary table */
+
+#cluster-process-group-summary-dialog {
+    display: none;
+    width: 778px;
+    height: 450px;
+    z-index: 1301;
+}
+
+#cluster-process-group-summary-table {
+    width: 758px;
+    height: 300px;
+    border-bottom: 1px solid #666;
+}
+
+#cluster-process-group-summary-header {
+    height: 26px;
+    color: #666;
+    font-weight: normal;
+    margin-bottom: 1px;
+}
+
+#cluster-process-group-refresh-button {
+    height: 24px;
+    width: 26px;
+    float: left;
+}
+
+#cluster-process-group-summary-last-refreshed-container {
+    float: left;
+    margin-top: 6px;
+    margin-left: 3px;
+    -webkit-user-select: none;
+    -moz-user-select: none;
+}
+
+#cluster-process-group-summary-last-refreshed {
+    font-weight: bold;
+}
+
+#cluster-process-group-summary-loading-container {
+    float: left;
+    width: 16px;
+    height: 16px;
+    background-color: transparent;
+    margin-top: 4px;
+    margin-left: 3px;
+}
+
+#cluster-process-group-details-container {
+    position: absolute;
+    right: 35px;
+}
+
+#cluster-process-group-icon {
+    background-image: url(../images/iconProcessGroup.png);
+    width: 29px;
+    height: 20px;
+    float: left;
+    margin-right: 5px;
+    margin-top: 1px;
+}
+
+#cluster-process-group-details {
+    float: left;
+}
+
+#cluster-process-group-name {
+    margin-bottom: 2px;
+    color: #000;
+    font-weight: bold;
+    white-space: nowrap;
+    overflow: hidden;
+    width: 200px;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/ff831dc3/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconProcessGroup.png
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconProcessGroup.png b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconProcessGroup.png
new file mode 100644
index 0000000..4ff5ac5
Binary files /dev/null and b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/images/iconProcessGroup.png differ