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 2016/04/04 18:28:50 UTC

[14/18] nifi git commit: NIFI-1563: - Federate requests and merge responses from nodes instead of storing bulletins and stats at NCM - Updating UI to support restructured status history DTO. - Return 'Insufficient History' message if aggregate stats don'

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
index dc17c21..b9bfd00 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java
@@ -14,38 +14,35 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.nifi.web.api.dto.status;
 
-import com.wordnik.swagger.annotations.ApiModelProperty;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
 import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 
-/**
- * DTO for serializing the status of a connection.
- */
-@XmlType(name = "connectionStatus")
-public class ConnectionStatusDTO {
+import com.wordnik.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
 
+@XmlType(name = "connectionStatus")
+public class ConnectionStatusDTO implements Cloneable {
     private String id;
     private String groupId;
     private String name;
-    private String input;
-    private String queuedCount;
-    private String queuedSize;
-    private String queued;
-    private String output;
+    private Date statsLastRefreshed;
 
     private String sourceId;
     private String sourceName;
     private String destinationId;
     private String destinationName;
 
-    /* getters / setters */
-    /**
-     * @return The connection id
-     */
-    @ApiModelProperty(
-            value = "The id of the connection."
-    )
+    private ConnectionStatusSnapshotDTO aggregateSnapshot;
+    private List<NodeConnectionStatusSnapshotDTO> nodeSnapshots;
+
+    @ApiModelProperty("The ID of the connection")
     public String getId() {
         return id;
     }
@@ -54,26 +51,16 @@ public class ConnectionStatusDTO {
         this.id = id;
     }
 
-    /**
-     * @return the ID of the Process Group to which this connection belongs.
-     */
-    @ApiModelProperty(
-            value = "The id of the process group the connection belongs to."
-    )
+    @ApiModelProperty("The ID of the Process Group that the connection belongs to")
     public String getGroupId() {
         return groupId;
     }
 
-    public void setGroupId(final String groupId) {
+    public void setGroupId(String groupId) {
         this.groupId = groupId;
     }
 
-    /**
-     * @return name of this connection
-     */
-    @ApiModelProperty(
-            value = "The name of the connection."
-    )
+    @ApiModelProperty("The name of the connection")
     public String getName() {
         return name;
     }
@@ -82,54 +69,7 @@ public class ConnectionStatusDTO {
         this.name = name;
     }
 
-    /**
-     * @return total count of flow files that are queued
-     */
-    @ApiModelProperty(
-            value = "The number of flowfiles that are queued."
-    )
-    public String getQueuedCount() {
-        return queuedCount;
-    }
-
-    public void setQueuedCount(String queuedCount) {
-        this.queuedCount = queuedCount;
-    }
-
-    /**
-     * @return total size of flow files that are queued
-     */
-    @ApiModelProperty(
-            value = "The total size of flowfiles that are queued formatted."
-    )
-    public String getQueuedSize() {
-        return queuedSize;
-    }
-
-    public void setQueuedSize(String queuedSize) {
-        this.queuedSize = queuedSize;
-    }
-
-    /**
-     * @return The total count and size of queued flow files
-     */
-    @ApiModelProperty(
-            value = "The total count and size of queued flowfiles formatted."
-    )
-    public String getQueued() {
-        return queued;
-    }
-
-    public void setQueued(String queued) {
-        this.queued = queued;
-    }
-
-    /**
-     * @return id of the source of this connection
-     */
-    @ApiModelProperty(
-            value = "The id of the source of the connection."
-    )
+    @ApiModelProperty("The ID of the source component")
     public String getSourceId() {
         return sourceId;
     }
@@ -138,12 +78,7 @@ public class ConnectionStatusDTO {
         this.sourceId = sourceId;
     }
 
-    /**
-     * @return name of the source of this connection
-     */
-    @ApiModelProperty(
-            value = "The name of the source of the connection."
-    )
+    @ApiModelProperty("The name of the source component")
     public String getSourceName() {
         return sourceName;
     }
@@ -152,12 +87,7 @@ public class ConnectionStatusDTO {
         this.sourceName = sourceName;
     }
 
-    /**
-     * @return id of the destination of this connection
-     */
-    @ApiModelProperty(
-            value = "The id of the destination of the connection."
-    )
+    @ApiModelProperty("The ID of the destination component")
     public String getDestinationId() {
         return destinationId;
     }
@@ -166,12 +96,7 @@ public class ConnectionStatusDTO {
         this.destinationId = destinationId;
     }
 
-    /**
-     * @return name of the destination of this connection
-     */
-    @ApiModelProperty(
-            value = "The name of the destination of the connection."
-    )
+    @ApiModelProperty("The name of the destination component")
     public String getDestinationName() {
         return destinationName;
     }
@@ -180,32 +105,53 @@ public class ConnectionStatusDTO {
         this.destinationName = destinationName;
     }
 
-    /**
-     * @return input for this connection
-     */
-    @ApiModelProperty(
-            value = "The input count/size for the connection in the last 5 minutes."
-    )
-    public String getInput() {
-        return input;
+    @ApiModelProperty("The status snapshot that represents the aggregate stats of the cluster")
+    public ConnectionStatusSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
     }
 
-    public void setInput(String input) {
-        this.input = input;
+    public void setAggregateSnapshot(ConnectionStatusSnapshotDTO aggregateSnapshot) {
+        this.aggregateSnapshot = aggregateSnapshot;
     }
 
-    /**
-     * @return output for this connection
-     */
-    @ApiModelProperty(
-            value = "The output count/sie for the connection in the last 5 minutes."
-    )
-    public String getOutput() {
-        return output;
+    @ApiModelProperty("A list of status snapshots for each node")
+    public List<NodeConnectionStatusSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
     }
 
-    public void setOutput(String output) {
-        this.output = output;
+    public void setNodeSnapshots(List<NodeConnectionStatusSnapshotDTO> nodeSnapshots) {
+        this.nodeSnapshots = nodeSnapshots;
     }
 
+    @XmlJavaTypeAdapter(TimeAdapter.class)
+    @ApiModelProperty("The timestamp of when the stats were last refreshed")
+    public Date getStatsLastRefreshed() {
+        return statsLastRefreshed;
+    }
+
+    public void setStatsLastRefreshed(Date statsLastRefreshed) {
+        this.statsLastRefreshed = statsLastRefreshed;
+    }
+
+    @Override
+    public ConnectionStatusDTO clone() {
+        final ConnectionStatusDTO other = new ConnectionStatusDTO();
+        other.setDestinationId(getDestinationId());
+        other.setDestinationName(getDestinationName());
+        other.setGroupId(getGroupId());
+        other.setId(getId());
+        other.setName(getName());
+        other.setSourceId(getSourceId());
+        other.setSourceName(getSourceName());
+        other.setAggregateSnapshot(getAggregateSnapshot().clone());
+
+        final List<NodeConnectionStatusSnapshotDTO> nodeStatuses = getNodeSnapshots();
+        final List<NodeConnectionStatusSnapshotDTO> nodeStatusClones = new ArrayList<>(nodeStatuses.size());
+        for (final NodeConnectionStatusSnapshotDTO nodeStatus : nodeStatuses) {
+            nodeStatusClones.add(nodeStatus.clone());
+        }
+        other.setNodeSnapshots(nodeStatusClones);
+
+        return other;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
new file mode 100644
index 0000000..928fc71
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusSnapshotDTO.java
@@ -0,0 +1,279 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ * DTO for serializing the status of a connection.
+ */
+@XmlType(name = "connectionStatusSnapshot")
+public class ConnectionStatusSnapshotDTO implements Cloneable {
+
+    private String id;
+    private String groupId;
+    private String name;
+
+    private String sourceId;
+    private String sourceName;
+    private String destinationId;
+    private String destinationName;
+
+    private Integer flowFilesIn = 0;
+    private Long bytesIn = 0L;
+    private String input;
+    private Integer flowFilesOut = 0;
+    private Long bytesOut = 0L;
+    private String output;
+    private Integer flowFilesQueued = 0;
+    private Long bytesQueued = 0L;
+    private String queued;
+    private String queuedSize;
+    private String queuedCount;
+
+    /* getters / setters */
+    /**
+     * @return The connection id
+     */
+    @ApiModelProperty("The id of the connection.")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return the ID of the Process Group to which this connection belongs.
+     */
+    @ApiModelProperty("The id of the process group the connection belongs to.")
+    public String getGroupId() {
+        return groupId;
+    }
+
+    public void setGroupId(final String groupId) {
+        this.groupId = groupId;
+    }
+
+    /**
+     * @return name of this connection
+     */
+    @ApiModelProperty("The name of the connection.")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return total count of flow files that are queued
+     */
+    @ApiModelProperty("The number of flowfiles that are queued, pretty printed.")
+    public String getQueuedCount() {
+        return queuedCount;
+    }
+
+    public void setQueuedCount(String queuedCount) {
+        this.queuedCount = queuedCount;
+    }
+
+
+    /**
+     * @return total size of flow files that are queued
+     */
+    @ApiModelProperty("The total size of flowfiles that are queued formatted.")
+    public String getQueuedSize() {
+        return queuedSize;
+    }
+
+
+    public void setInput(String input) {
+        this.input = input;
+    }
+
+    public void setOutput(String output) {
+        this.output = output;
+    }
+
+    public void setQueued(String queued) {
+        this.queued = queued;
+    }
+
+    public void setQueuedSize(String queuedSize) {
+        this.queuedSize = queuedSize;
+    }
+
+    /**
+     * @return The total count and size of queued flow files
+     */
+    @ApiModelProperty("The total count and size of queued flowfiles formatted.")
+    public String getQueued() {
+        return queued;
+    }
+
+
+    /**
+     * @return id of the source of this connection
+     */
+    @ApiModelProperty("The id of the source of the connection.")
+    public String getSourceId() {
+        return sourceId;
+    }
+
+    public void setSourceId(String sourceId) {
+        this.sourceId = sourceId;
+    }
+
+    /**
+     * @return name of the source of this connection
+     */
+    @ApiModelProperty("The name of the source of the connection.")
+    public String getSourceName() {
+        return sourceName;
+    }
+
+    public void setSourceName(String sourceName) {
+        this.sourceName = sourceName;
+    }
+
+    /**
+     * @return id of the destination of this connection
+     */
+    @ApiModelProperty("The id of the destination of the connection.")
+    public String getDestinationId() {
+        return destinationId;
+    }
+
+    public void setDestinationId(String destinationId) {
+        this.destinationId = destinationId;
+    }
+
+    /**
+     * @return name of the destination of this connection
+     */
+    @ApiModelProperty("The name of the destination of the connection.")
+    public String getDestinationName() {
+        return destinationName;
+    }
+
+    public void setDestinationName(String destinationName) {
+        this.destinationName = destinationName;
+    }
+
+    /**
+     * @return input for this connection
+     */
+    @ApiModelProperty("The input count/size for the connection in the last 5 minutes, pretty printed.")
+    public String getInput() {
+        return input;
+    }
+
+
+    /**
+     * @return output for this connection
+     */
+    @ApiModelProperty("The output count/sie for the connection in the last 5 minutes, pretty printed.")
+    public String getOutput() {
+        return output;
+    }
+
+
+    @ApiModelProperty("The number of FlowFiles that have come into the connection in the last 5 minutes.")
+    public Integer getFlowFilesIn() {
+        return flowFilesIn;
+    }
+
+    public void setFlowFilesIn(Integer flowFilesIn) {
+        this.flowFilesIn = flowFilesIn;
+    }
+
+    @ApiModelProperty("The size of the FlowFiles that have come into the connection in the last 5 minutes.")
+    public Long getBytesIn() {
+        return bytesIn;
+    }
+
+    public void setBytesIn(Long bytesIn) {
+        this.bytesIn = bytesIn;
+    }
+
+    @ApiModelProperty("The number of FlowFiles that have left the connection in the last 5 minutes.")
+    public Integer getFlowFilesOut() {
+        return flowFilesOut;
+    }
+
+    public void setFlowFilesOut(Integer flowFilesOut) {
+        this.flowFilesOut = flowFilesOut;
+    }
+
+    @ApiModelProperty("The number of bytes that have left the connection in the last 5 minutes.")
+    public Long getBytesOut() {
+        return bytesOut;
+    }
+
+    public void setBytesOut(Long bytesOut) {
+        this.bytesOut = bytesOut;
+    }
+
+    @ApiModelProperty("The number of FlowFiles that are currently queued in the connection.")
+    public Integer getFlowFilesQueued() {
+        return flowFilesQueued;
+    }
+
+    public void setFlowFilesQueued(Integer flowFilesQueued) {
+        this.flowFilesQueued = flowFilesQueued;
+    }
+
+    @ApiModelProperty("The size of the FlowFiles that are currently queued in the connection.")
+    public Long getBytesQueued() {
+        return bytesQueued;
+    }
+
+    public void setBytesQueued(Long bytesQueued) {
+        this.bytesQueued = bytesQueued;
+    }
+
+
+    @Override
+    public ConnectionStatusSnapshotDTO clone() {
+        final ConnectionStatusSnapshotDTO other = new ConnectionStatusSnapshotDTO();
+        other.setDestinationId(getDestinationId());
+        other.setDestinationName(getDestinationName());
+        other.setGroupId(getGroupId());
+        other.setId(getId());
+        other.setName(getName());
+        other.setSourceId(getSourceId());
+        other.setSourceName(getSourceName());
+
+        other.setFlowFilesIn(getFlowFilesIn());
+        other.setBytesIn(getBytesIn());
+        other.setInput(getInput());
+        other.setFlowFilesOut(getFlowFilesOut());
+        other.setBytesOut(getBytesOut());
+        other.setOutput(getOutput());
+        other.setFlowFilesQueued(getFlowFilesQueued());
+        other.setBytesQueued(getBytesQueued());
+        other.setQueued(getQueued());
+        other.setQueuedCount(getQueuedCount());
+        other.setQueuedSize(getQueuedSize());
+
+        return other;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
index 5d5eddf..03e2124 100644
--- a/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java
@@ -17,6 +17,8 @@
 package org.apache.nifi.web.api.dto.status;
 
 import com.wordnik.swagger.annotations.ApiModelProperty;
+
+import java.util.ArrayList;
 import java.util.List;
 import javax.xml.bind.annotation.XmlType;
 import org.apache.nifi.web.api.dto.BulletinDTO;
@@ -25,19 +27,25 @@ import org.apache.nifi.web.api.dto.BulletinDTO;
  * The status of this NiFi controller.
  */
 @XmlType(name = "controllerStatus")
-public class ControllerStatusDTO {
+public class ControllerStatusDTO implements Cloneable {
 
-    private Integer activeThreadCount;
+    private Integer activeThreadCount = 0;
     private String queued;
+    private Integer flowFilesQueued = 0;
+    private Long bytesQueued = 0L;
+
     private String connectedNodes;
+    private Integer connectedNodeCount = 0;
+    private Integer totalNodeCount = 0;
+
     private Boolean hasPendingAccounts;
 
-    private Integer runningCount;
-    private Integer stoppedCount;
-    private Integer invalidCount;
-    private Integer disabledCount;
-    private Integer activeRemotePortCount;
-    private Integer inactiveRemotePortCount;
+    private Integer runningCount = 0;
+    private Integer stoppedCount = 0;
+    private Integer invalidCount = 0;
+    private Integer disabledCount = 0;
+    private Integer activeRemotePortCount = 0;
+    private Integer inactiveRemotePortCount = 0;
 
     private List<BulletinDTO> bulletins;
     private List<BulletinDTO> controllerServiceBulletins;
@@ -48,9 +56,7 @@ public class ControllerStatusDTO {
      *
      * @return The active thread count
      */
-    @ApiModelProperty(
-            value = "The number of active threads in the NiFi."
-    )
+    @ApiModelProperty("The number of active threads in the NiFi.")
     public Integer getActiveThreadCount() {
         return activeThreadCount;
     }
@@ -62,9 +68,7 @@ public class ControllerStatusDTO {
     /**
      * @return queue for the controller
      */
-    @ApiModelProperty(
-            value = "The number of flowfilew queued in the NiFi."
-    )
+    @ApiModelProperty("The number of flowfilew queued in the NiFi.")
     public String getQueued() {
         return queued;
     }
@@ -77,9 +81,7 @@ public class ControllerStatusDTO {
      * @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."
-    )
+    @ApiModelProperty("When clustered, reports the number of nodes connected vs the number of nodes in the cluster.")
     public String getConnectedNodes() {
         return connectedNodes;
     }
@@ -91,9 +93,7 @@ public class ControllerStatusDTO {
     /**
      * @return System bulletins to be reported to the user
      */
-    @ApiModelProperty(
-            value = "System level bulletins to be reported to the user."
-    )
+    @ApiModelProperty("System level bulletins to be reported to the user.")
     public List<BulletinDTO> getBulletins() {
         return bulletins;
     }
@@ -105,9 +105,7 @@ public class ControllerStatusDTO {
     /**
      * @return Controller service bulletins to be reported to the user
      */
-    @ApiModelProperty(
-            value = "Controller service bulletins to be reported to the user."
-    )
+    @ApiModelProperty("Controller service bulletins to be reported to the user.")
     public List<BulletinDTO> getControllerServiceBulletins() {
         return controllerServiceBulletins;
     }
@@ -119,9 +117,7 @@ public class ControllerStatusDTO {
     /**
      * @return Reporting task bulletins to be reported to the user
      */
-    @ApiModelProperty(
-            value = "Reporting task bulletins to be reported to the user."
-    )
+    @ApiModelProperty("Reporting task bulletins to be reported to the user.")
     public List<BulletinDTO> getReportingTaskBulletins() {
         return reportingTaskBulletins;
     }
@@ -133,9 +129,7 @@ public class ControllerStatusDTO {
     /**
      * @return whether or not there are pending user requests
      */
-    @ApiModelProperty(
-            value = "Whether there are any pending user account requests."
-    )
+    @ApiModelProperty("Whether there are any pending user account requests.")
     public Boolean getHasPendingAccounts() {
         return hasPendingAccounts;
     }
@@ -147,9 +141,7 @@ public class ControllerStatusDTO {
     /**
      * @return number of running components in this controller
      */
-    @ApiModelProperty(
-            value = "The number of running components in the NiFi."
-    )
+    @ApiModelProperty("The number of running components in the NiFi.")
     public Integer getRunningCount() {
         return runningCount;
     }
@@ -161,9 +153,7 @@ public class ControllerStatusDTO {
     /**
      * @return number of stopped components in this controller
      */
-    @ApiModelProperty(
-            value = "The number of stopped components in the NiFi."
-    )
+    @ApiModelProperty("The number of stopped components in the NiFi.")
     public Integer getStoppedCount() {
         return stoppedCount;
     }
@@ -175,9 +165,7 @@ public class ControllerStatusDTO {
     /**
      * @return number of invalid components in this controller
      */
-    @ApiModelProperty(
-            value = "The number of invalid components in the NiFi."
-    )
+    @ApiModelProperty("The number of invalid components in the NiFi.")
     public Integer getInvalidCount() {
         return invalidCount;
     }
@@ -189,9 +177,7 @@ public class ControllerStatusDTO {
     /**
      * @return number of disabled components in this controller
      */
-    @ApiModelProperty(
-            value = "The number of disabled components in the NiFi."
-    )
+    @ApiModelProperty("The number of disabled components in the NiFi.")
     public Integer getDisabledCount() {
         return disabledCount;
     }
@@ -203,9 +189,7 @@ public class ControllerStatusDTO {
     /**
      * @return number of active remote ports in this controller
      */
-    @ApiModelProperty(
-            value = "The number of active remote ports in the NiFi."
-    )
+    @ApiModelProperty("The number of active remote ports in the NiFi.")
     public Integer getActiveRemotePortCount() {
         return activeRemotePortCount;
     }
@@ -217,9 +201,7 @@ public class ControllerStatusDTO {
     /**
      * @return number of inactive remote ports in this controller
      */
-    @ApiModelProperty(
-            value = "The number of inactive remote ports in the NiFi."
-    )
+    @ApiModelProperty("The number of inactive remote ports in the NiFi.")
     public Integer getInactiveRemotePortCount() {
         return inactiveRemotePortCount;
     }
@@ -228,4 +210,62 @@ public class ControllerStatusDTO {
         this.inactiveRemotePortCount = inactiveRemotePortCount;
     }
 
+    @ApiModelProperty("The number of FlowFiles queued across the entire flow")
+    public Integer getFlowFilesQueued() {
+        return flowFilesQueued;
+    }
+
+    public void setFlowFilesQueued(Integer flowFilesQueued) {
+        this.flowFilesQueued = flowFilesQueued;
+    }
+
+    @ApiModelProperty("The size of the FlowFiles queued across the entire flow")
+    public Long getBytesQueued() {
+        return bytesQueued;
+    }
+
+    public void setBytesQueued(Long bytesQueued) {
+        this.bytesQueued = bytesQueued;
+    }
+
+    @ApiModelProperty("The number of nodes that are currently connected to the cluster")
+    public Integer getConnectedNodeCount() {
+        return connectedNodeCount;
+    }
+
+    public void setConnectedNodeCount(Integer connectedNodeCount) {
+        this.connectedNodeCount = connectedNodeCount;
+    }
+
+    @ApiModelProperty("The number of nodes in the cluster, regardless of whether or not they are connected")
+    public Integer getTotalNodeCount() {
+        return totalNodeCount;
+    }
+
+    public void setTotalNodeCount(Integer totalNodeCount) {
+        this.totalNodeCount = totalNodeCount;
+    }
+
+    @Override
+    public ControllerStatusDTO clone() {
+        final ControllerStatusDTO other = new ControllerStatusDTO();
+        other.setActiveThreadCount(getActiveThreadCount());
+        other.setQueued(getQueued());
+        other.setFlowFilesQueued(getFlowFilesQueued());
+        other.setBytesQueued(getBytesQueued());
+        other.setConnectedNodes(getConnectedNodes());
+        other.setConnectedNodeCount(getConnectedNodeCount());
+        other.setTotalNodeCount(getTotalNodeCount());
+        other.setHasPendingAccounts(getHasPendingAccounts());
+        other.setRunningCount(getRunningCount());
+        other.setStoppedCount(getStoppedCount());
+        other.setInvalidCount(getInvalidCount());
+        other.setDisabledCount(getDisabledCount());
+        other.setActiveRemotePortCount(getActiveRemotePortCount());
+        other.setInactiveRemotePortCount(getInactiveRemotePortCount());
+        other.setBulletins(getBulletins() == null ? null : new ArrayList<>(getBulletins()));
+        other.setControllerServiceBulletins(getControllerServiceBulletins() == null ? null : new ArrayList<>(getControllerServiceBulletins()));
+        other.setReportingTaskBulletins(getReportingTaskBulletins() == null ? null : new ArrayList<>(getReportingTaskBulletins()));
+        return other;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java
deleted file mode 100644
index bcc4045..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web.api.dto.status;
-
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the connection status for a particular node.
- */
-@XmlType(name = "nodeConnectionStatus")
-public class NodeConnectionStatusDTO {
-
-    private NodeDTO node;
-    private ConnectionStatusDTO connectionStatus;
-
-    /**
-     * @return the node
-     */
-    @ApiModelProperty(
-            value = "The node."
-    )
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * @return connection's status
-     */
-    @ApiModelProperty(
-            value = "The connection status from the node."
-    )
-    public ConnectionStatusDTO getConnectionStatus() {
-        return connectionStatus;
-    }
-
-    public void setConnectionStatus(ConnectionStatusDTO connectionStatus) {
-        this.connectionStatus = connectionStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusSnapshotDTO.java
new file mode 100644
index 0000000..41f7433
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusSnapshotDTO.java
@@ -0,0 +1,78 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "nodeConnectionStatusSnapshot")
+public class NodeConnectionStatusSnapshotDTO implements Cloneable {
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private ConnectionStatusSnapshotDTO statusSnapshot;
+
+    @ApiModelProperty("The unique ID that identifies the node")
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @ApiModelProperty("The API address of the node")
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    @ApiModelProperty("The API port used to communicate with the node")
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer apiPort) {
+        this.apiPort = apiPort;
+    }
+
+    @ApiModelProperty("The connection status snapshot from the node.")
+    public ConnectionStatusSnapshotDTO getStatusSnapshot() {
+        return statusSnapshot;
+    }
+
+    public void setStatusSnapshot(ConnectionStatusSnapshotDTO statusSnapshot) {
+        this.statusSnapshot = statusSnapshot;
+    }
+
+    @Override
+    public NodeConnectionStatusSnapshotDTO clone() {
+        final NodeConnectionStatusSnapshotDTO other = new NodeConnectionStatusSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setStatusSnapshot(getStatusSnapshot().clone());
+        return other;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusSnapshotDTO.java
new file mode 100644
index 0000000..a8dc1ff
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusSnapshotDTO.java
@@ -0,0 +1,77 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "nodePortStatusSnapshot")
+public class NodePortStatusSnapshotDTO implements Cloneable {
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private PortStatusSnapshotDTO statusSnapshot;
+
+    @ApiModelProperty("The unique ID that identifies the node")
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @ApiModelProperty("The API address of the node")
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    @ApiModelProperty("The API port used to communicate with the node")
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer apiPort) {
+        this.apiPort = apiPort;
+    }
+
+    @ApiModelProperty("The port status snapshot from the node.")
+    public PortStatusSnapshotDTO getStatusSnapshot() {
+        return statusSnapshot;
+    }
+
+    public void setStatusSnapshot(PortStatusSnapshotDTO status) {
+        this.statusSnapshot = status;
+    }
+
+    @Override
+    public NodePortStatusSnapshotDTO clone() {
+        final NodePortStatusSnapshotDTO other = new NodePortStatusSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setStatusSnapshot(getStatusSnapshot().clone());
+        return other;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusDTO.java
deleted file mode 100644
index 96c59fa..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusDTO.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web.api.dto.status;
-
-import com.wordnik.swagger.annotations.ApiModelProperty;
-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
-     */
-    @ApiModelProperty(
-            value = "The node."
-    )
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * The process group's status.
-     *
-     * @return The process group status
-     */
-    @ApiModelProperty(
-            value = "The process group status from the node."
-    )
-    public ProcessGroupStatusDTO getProcessGroupStatus() {
-        return processGroupStatus;
-    }
-
-    public void setProcessGroupStatus(ProcessGroupStatusDTO processGroupStatus) {
-        this.processGroupStatus = processGroupStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusSnapshotDTO.java
new file mode 100644
index 0000000..854b077
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessGroupStatusSnapshotDTO.java
@@ -0,0 +1,76 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+public class NodeProcessGroupStatusSnapshotDTO implements Cloneable {
+
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private ProcessGroupStatusSnapshotDTO statusSnapshot;
+
+    @ApiModelProperty("The unique ID that identifies the node")
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @ApiModelProperty("The API address of the node")
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    @ApiModelProperty("The API port used to communicate with the node")
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer apiPort) {
+        this.apiPort = apiPort;
+    }
+
+    @ApiModelProperty("The process group status snapshot from the node.")
+    public ProcessGroupStatusSnapshotDTO getStatusSnapshot() {
+        return statusSnapshot;
+    }
+
+    public void setStatusSnapshot(ProcessGroupStatusSnapshotDTO statusSnapshot) {
+        this.statusSnapshot = statusSnapshot;
+    }
+
+    @Override
+    public NodeProcessGroupStatusSnapshotDTO clone() {
+        final NodeProcessGroupStatusSnapshotDTO other = new NodeProcessGroupStatusSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setStatusSnapshot(getStatusSnapshot().clone());
+        return other;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java
deleted file mode 100644
index 8c8f604..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web.api.dto.status;
-
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the processor status for a particular node.
- */
-@XmlType(name = "nodeProcessorStatus")
-public class NodeProcessorStatusDTO {
-
-    private NodeDTO node;
-    private ProcessorStatusDTO processorStatus;
-
-    /**
-     * @return the node
-     */
-    @ApiModelProperty(
-            value = "The node."
-    )
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * @return processor's status
-     */
-    @ApiModelProperty(
-            value = "The processor status from the node."
-    )
-    public ProcessorStatusDTO getProcessorStatus() {
-        return processorStatus;
-    }
-
-    public void setProcessorStatus(ProcessorStatusDTO processorStatus) {
-        this.processorStatus = processorStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusSnapshotDTO.java
new file mode 100644
index 0000000..5ea4697
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusSnapshotDTO.java
@@ -0,0 +1,80 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+/**
+ * DTO for serializing the processor status for a particular node.
+ */
+@XmlType(name = "nodeProcessorStatusSnapshot")
+public class NodeProcessorStatusSnapshotDTO implements Cloneable {
+
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private ProcessorStatusSnapshotDTO statusSnapshot;
+
+    @ApiModelProperty("The unique ID that identifies the node")
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @ApiModelProperty("The API address of the node")
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    @ApiModelProperty("The API port used to communicate with the node")
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer apiPort) {
+        this.apiPort = apiPort;
+    }
+
+    @ApiModelProperty("The processor status snapshot from the node.")
+    public ProcessorStatusSnapshotDTO getStatusSnapshot() {
+        return statusSnapshot;
+    }
+
+    public void setStatusSnapshot(ProcessorStatusSnapshotDTO processorStatusSnapshot) {
+        this.statusSnapshot = processorStatusSnapshot;
+    }
+
+    @Override
+    public NodeProcessorStatusSnapshotDTO clone() {
+        final NodeProcessorStatusSnapshotDTO other = new NodeProcessorStatusSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setStatusSnapshot(getStatusSnapshot().clone());
+        return other;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemotePortStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemotePortStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemotePortStatusSnapshotDTO.java
new file mode 100644
index 0000000..3cd93bc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemotePortStatusSnapshotDTO.java
@@ -0,0 +1,77 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "nodeRemotePortStatusSnapshot")
+public class NodeRemotePortStatusSnapshotDTO implements Cloneable {
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private RemotePortStatusDTO statusSnapshot;
+
+    @ApiModelProperty("The unique ID that identifies the node")
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @ApiModelProperty("The API address of the node")
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    @ApiModelProperty("The API port used to communicate with the node")
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer apiPort) {
+        this.apiPort = apiPort;
+    }
+
+    @ApiModelProperty("The remote port status snapshot from the node.")
+    public RemotePortStatusDTO getStatusSnapshot() {
+        return statusSnapshot;
+    }
+
+    public void setStatusSnapshot(RemotePortStatusDTO statusSnapshot) {
+        this.statusSnapshot = statusSnapshot;
+    }
+
+    @Override
+    public NodeRemotePortStatusSnapshotDTO clone() {
+        final NodeRemotePortStatusSnapshotDTO other = new NodeRemotePortStatusSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setStatusSnapshot(getStatusSnapshot().clone());
+        return other;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java
deleted file mode 100644
index 512b4c2..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web.api.dto.status;
-
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the remote process group status for a particular node.
- */
-@XmlType(name = "nodeRemoteProcessGroupStatus")
-public class NodeRemoteProcessGroupStatusDTO {
-
-    private NodeDTO node;
-    private RemoteProcessGroupStatusDTO remoteProcessGroupStatus;
-
-    /**
-     * @return the node
-     */
-    @ApiModelProperty(
-            value = "The node."
-    )
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * @return remote process group's status
-     */
-    @ApiModelProperty(
-            value = "The remote process group status from the node."
-    )
-    public RemoteProcessGroupStatusDTO getRemoteProcessGroupStatus() {
-        return remoteProcessGroupStatus;
-    }
-
-    public void setRemoteProcessGroupStatus(RemoteProcessGroupStatusDTO remoteProcessGroupStatus) {
-        this.remoteProcessGroupStatus = remoteProcessGroupStatus;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusSnapshotDTO.java
new file mode 100644
index 0000000..57658f2
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusSnapshotDTO.java
@@ -0,0 +1,77 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "nodeRemoteProcessGroupStatusSnapshot")
+public class NodeRemoteProcessGroupStatusSnapshotDTO implements Cloneable {
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private RemoteProcessGroupStatusSnapshotDTO statusSnapshot;
+
+    @ApiModelProperty("The unique ID that identifies the node")
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    @ApiModelProperty("The API address of the node")
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    @ApiModelProperty("The API port used to communicate with the node")
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer apiPort) {
+        this.apiPort = apiPort;
+    }
+
+    @ApiModelProperty("The remote process group status snapshot from the node.")
+    public RemoteProcessGroupStatusSnapshotDTO getStatusSnapshot() {
+        return statusSnapshot;
+    }
+
+    public void setStatusSnapshot(RemoteProcessGroupStatusSnapshotDTO statusSnapshot) {
+        this.statusSnapshot = statusSnapshot;
+    }
+
+    @Override
+    public NodeRemoteProcessGroupStatusSnapshotDTO clone() {
+        final NodeRemoteProcessGroupStatusSnapshotDTO other = new NodeRemoteProcessGroupStatusSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setStatusSnapshot(getStatusSnapshot().clone());
+        return other;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java
deleted file mode 100644
index 5cf9f41..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.web.api.dto.status;
-
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.NodeDTO;
-
-/**
- * DTO for serializing the status history for a particular node.
- */
-@XmlType(name = "nodeStatusHistory")
-public class NodeStatusHistoryDTO {
-
-    private NodeDTO node;
-    private StatusHistoryDTO statusHistory;
-
-    /**
-     * @return the node
-     */
-    @ApiModelProperty(
-            value = "The node."
-    )
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * @return processor status history
-     */
-    @ApiModelProperty(
-            value = "The processor status for each node."
-    )
-    public StatusHistoryDTO getStatusHistory() {
-        return statusHistory;
-    }
-
-    public void setStatusHistory(StatusHistoryDTO statusHistory) {
-        this.statusHistory = statusHistory;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusSnapshotsDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusSnapshotsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusSnapshotsDTO.java
new file mode 100644
index 0000000..f70da4a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusSnapshotsDTO.java
@@ -0,0 +1,85 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+import javax.xml.bind.annotation.XmlType;
+import java.util.List;
+
+/**
+ * DTO for serializing the status history of a single component across the cluster.
+ */
+@XmlType(name = "nodeStatusSnapshots")
+public class NodeStatusSnapshotsDTO {
+
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+    private List<StatusSnapshotDTO> statusSnapshots;
+
+    /**
+     * @return node's host/IP address
+     */
+    @ApiModelProperty(
+        value = "The node's host/ip address."
+    )
+    public String getAddress() {
+        return address;
+    }
+
+    public void setAddress(String address) {
+        this.address = address;
+    }
+
+    /**
+     * @return node ID
+     */
+    @ApiModelProperty(
+        value = "The id of the node."
+    )
+    public String getNodeId() {
+        return nodeId;
+    }
+
+    public void setNodeId(String nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    /**
+     * @return port the node is listening for API requests
+     */
+    @ApiModelProperty(
+        value = "The port the node is listening for API requests."
+    )
+    public Integer getApiPort() {
+        return apiPort;
+    }
+
+    public void setApiPort(Integer port) {
+        this.apiPort = port;
+    }
+
+    @ApiModelProperty("A list of StatusSnapshotDTO objects that provide the actual metric values for the component for this node.")
+    public List<StatusSnapshotDTO> getStatusSnapshots() {
+        return statusSnapshots;
+    }
+
+    public void setStatusSnapshots(List<StatusSnapshotDTO> statusSnapshots) {
+        this.statusSnapshots = statusSnapshots;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
index c1d95d0..5e44e71 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java
@@ -14,32 +14,31 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.nifi.web.api.dto.status;
 
-import com.wordnik.swagger.annotations.ApiModelProperty;
+import java.util.Date;
+import java.util.List;
+
 import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 
-/**
- * The status for a port in this NiFi.
- */
-@XmlType(name = "portStatus")
-public class PortStatusDTO extends StatusDTO {
+import com.wordnik.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
 
+@XmlType(name = "portStatus")
+public class PortStatusDTO {
     private String id;
     private String groupId;
     private String name;
-    private Integer activeThreadCount;
-    private String input;
-    private String output;
     private Boolean transmitting;
     private String runStatus;
+    private Date statsLastRefreshed;
 
-    /**
-     * @return whether this port has incoming or outgoing connections to a remote NiFi
-     */
-    @ApiModelProperty(
-            value = "Whether the port has incoming or outgoing connections to a remote NiFi."
-    )
+    private PortStatusSnapshotDTO aggregateSnapshot;
+    private List<NodePortStatusSnapshotDTO> nodeSnapshots;
+
+    @ApiModelProperty("Whether the port has incoming or outgoing connections to a remote NiFi.")
     public Boolean isTransmitting() {
         return transmitting;
     }
@@ -48,26 +47,8 @@ public class PortStatusDTO extends StatusDTO {
         this.transmitting = transmitting;
     }
 
-    /**
-     * @return the active thread count for this port
-     */
-    @ApiModelProperty(
-            value = "The active thread count for the port."
-    )
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
 
-    /**
-     * @return id of this port
-     */
-    @ApiModelProperty(
-            value = "The id of the port."
-    )
+    @ApiModelProperty("The id of the port.")
     public String getId() {
         return id;
     }
@@ -76,12 +57,8 @@ public class PortStatusDTO extends StatusDTO {
         this.id = id;
     }
 
-    /**
-     * @return id of the group this port resides in
-     */
-    @ApiModelProperty(
-            value = "The id of the parent process group of the port."
-    )
+
+    @ApiModelProperty("The id of the parent process group of the port.")
     public String getGroupId() {
         return groupId;
     }
@@ -90,12 +67,8 @@ public class PortStatusDTO extends StatusDTO {
         this.groupId = groupId;
     }
 
-    /**
-     * @return name of this port
-     */
-    @ApiModelProperty(
-            value = "The name of the port."
-    )
+
+    @ApiModelProperty("The name of the port.")
     public String getName() {
         return name;
     }
@@ -104,12 +77,8 @@ public class PortStatusDTO extends StatusDTO {
         this.name = name;
     }
 
-    /**
-     * @return run status of this port
-     */
-    @ApiModelProperty(
-            value = "The run status of the port."
-    )
+
+    @ApiModelProperty("The run status of the port.")
     public String getRunStatus() {
         return runStatus;
     }
@@ -118,32 +87,40 @@ public class PortStatusDTO extends StatusDTO {
         this.runStatus = runStatus;
     }
 
-    /**
-     * @return The total count and size of flow files that have been accepted in the last five minutes
-     */
-    @ApiModelProperty(
-            value = "The count/size of flowfiles that have been accepted in the last 5 minutes."
-    )
-    public String getInput() {
-        return input;
+    @ApiModelProperty("A status snapshot that represents the aggregate stats of all nodes in the cluster. If the NiFi instance is "
+        + "a standalone instance, rather than a cluster, this represents the stats of the single instance.")
+    public PortStatusSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
     }
 
-    public void setInput(String input) {
-        this.input = input;
+    public void setAggregateSnapshot(PortStatusSnapshotDTO aggregateSnapshot) {
+        this.aggregateSnapshot = aggregateSnapshot;
+    }
+
+    @ApiModelProperty("A status snapshot for each node in the cluster. If the NiFi instance is a standalone instance, rather than "
+        + "a cluster, this may be null.")
+    public List<NodePortStatusSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
+    }
+
+    public void setNodeSnapshots(List<NodePortStatusSnapshotDTO> nodeSnapshots) {
+        this.nodeSnapshots = nodeSnapshots;
     }
 
     /**
-     * @return The total count and size of flow files that have been processed in the last five minutes
+     * When the status for this process group was calculated.
+     *
+     * @return The the status was calculated
      */
+    @XmlJavaTypeAdapter(TimeAdapter.class)
     @ApiModelProperty(
-            value = "The count/size of flowfiles that have been processed in the last 5 minutes."
+        value = "The time the status for the process group was last refreshed."
     )
-    public String getOutput() {
-        return output;
+    public Date getStatsLastRefreshed() {
+        return statsLastRefreshed;
     }
 
-    public void setOutput(String output) {
-        this.output = output;
+    public void setStatsLastRefreshed(Date statsLastRefreshed) {
+        this.statsLastRefreshed = statsLastRefreshed;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusSnapshotDTO.java
new file mode 100644
index 0000000..b5f11e5
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusSnapshotDTO.java
@@ -0,0 +1,194 @@
+/*
+ * 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 com.wordnik.swagger.annotations.ApiModelProperty;
+
+/**
+ * The status for a port in this NiFi.
+ */
+@XmlType(name = "portStatusSnapshot")
+public class PortStatusSnapshotDTO implements Cloneable {
+
+    private String id;
+    private String groupId;
+    private String name;
+
+    private Integer activeThreadCount = 0;
+    private Integer flowFilesIn = 0;
+    private Long bytesIn = 0L;
+    private String input;
+    private Integer flowFilesOut = 0;
+    private Long bytesOut = 0L;
+    private String output;
+
+    private Boolean transmitting;
+    private String runStatus;
+
+    /**
+     * @return whether this port has incoming or outgoing connections to a remote NiFi
+     */
+    @ApiModelProperty("Whether the port has incoming or outgoing connections to a remote NiFi.")
+    public Boolean isTransmitting() {
+        return transmitting;
+    }
+
+    public void setTransmitting(Boolean transmitting) {
+        this.transmitting = transmitting;
+    }
+
+    /**
+     * @return the active thread count for this port
+     */
+    @ApiModelProperty("The active thread count for the port.")
+    public Integer getActiveThreadCount() {
+        return activeThreadCount;
+    }
+
+    public void setActiveThreadCount(Integer activeThreadCount) {
+        this.activeThreadCount = activeThreadCount;
+    }
+
+    /**
+     * @return id of this port
+     */
+    @ApiModelProperty("The id of the port.")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return id of the group this port resides in
+     */
+    @ApiModelProperty("The id of the parent process group of the port.")
+    public String getGroupId() {
+        return groupId;
+    }
+
+    public void setGroupId(String groupId) {
+        this.groupId = groupId;
+    }
+
+    /**
+     * @return name of this port
+     */
+    @ApiModelProperty("The name of the port.")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return run status of this port
+     */
+    @ApiModelProperty("The run status of the port.")
+    public String getRunStatus() {
+        return runStatus;
+    }
+
+    public void setRunStatus(String runStatus) {
+        this.runStatus = runStatus;
+    }
+
+    /**
+     * @return The total count and size of flow files that have been accepted in the last five minutes
+     */
+    @ApiModelProperty("The count/size of flowfiles that have been accepted in the last 5 minutes.")
+    public String getInput() {
+        return input;
+    }
+
+    public void setInput(String input) {
+        this.input = input;
+    }
+
+    /**
+     * @return The total count and size of flow files that have been processed in the last five minutes
+     */
+    @ApiModelProperty("The count/size of flowfiles that have been processed in the last 5 minutes.")
+    public String getOutput() {
+        return output;
+    }
+
+    public void setOutput(String output) {
+        this.output = output;
+    }
+
+    @ApiModelProperty("The number of FlowFiles that have been accepted in the last 5 minutes.")
+    public Integer getFlowFilesIn() {
+        return flowFilesIn;
+    }
+
+    public void setFlowFilesIn(Integer flowFilesIn) {
+        this.flowFilesIn = flowFilesIn;
+    }
+
+    @ApiModelProperty("The size of hte FlowFiles that have been accepted in the last 5 minutes.")
+    public Long getBytesIn() {
+        return bytesIn;
+    }
+
+    public void setBytesIn(Long bytesIn) {
+        this.bytesIn = bytesIn;
+    }
+
+    @ApiModelProperty("The number of FlowFiles that have been processed in the last 5 minutes.")
+    public Integer getFlowFilesOut() {
+        return flowFilesOut;
+    }
+
+    public void setFlowFilesOut(Integer flowFilesOut) {
+        this.flowFilesOut = flowFilesOut;
+    }
+
+    @ApiModelProperty("The number of bytes that have been processed in the last 5 minutes.")
+    public Long getBytesOut() {
+        return bytesOut;
+    }
+
+    public void setBytesOut(Long bytesOut) {
+        this.bytesOut = bytesOut;
+    }
+
+    @Override
+    public PortStatusSnapshotDTO clone() {
+        final PortStatusSnapshotDTO other = new PortStatusSnapshotDTO();
+        other.setId(getId());
+        other.setGroupId(getGroupId());
+        other.setName(getName());
+        other.setActiveThreadCount(getActiveThreadCount());
+        other.setFlowFilesIn(getFlowFilesIn());
+        other.setBytesIn(getBytesIn());
+        other.setFlowFilesOut(getFlowFilesOut());
+        other.setBytesOut(getBytesOut());
+        other.setTransmitting(isTransmitting());
+        other.setRunStatus(getRunStatus());
+        other.setInput(getInput());
+        other.setOutput(getOutput());
+
+        return other;
+    }
+}