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:49 UTC

[13/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/ProcessGroupStatusDTO.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/ProcessGroupStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
index 3ebae8f..1df212a 100644
--- a/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-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java
@@ -14,51 +14,28 @@
  * 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.Collection;
 import java.util.Date;
+import java.util.List;
+
 import javax.xml.bind.annotation.XmlType;
 import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
 import org.apache.nifi.web.api.dto.util.TimeAdapter;
 
-/**
- * The status for a process group in this NiFi.
- */
 @XmlType(name = "processGroupStatus")
-public class ProcessGroupStatusDTO extends StatusDTO {
-
+public class ProcessGroupStatusDTO implements Cloneable {
     private String id;
     private String name;
-    private Collection<ConnectionStatusDTO> connectionStatus;
-    private Collection<ProcessorStatusDTO> processorStatus;
-    private Collection<ProcessGroupStatusDTO> processGroupStatus;
-    private Collection<RemoteProcessGroupStatusDTO> remoteProcessGroupStatus;
-    private Collection<PortStatusDTO> inputPortStatus;
-    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;
 
-    /**
-     * The id for the process group.
-     *
-     * @return The id for the process group
-     */
-    @ApiModelProperty(
-            value = "The id of the process group."
-    )
+    private ProcessGroupStatusSnapshotDTO aggregateSnapshot;
+    private List<NodeProcessGroupStatusSnapshotDTO> nodeSnapshots;
+
+    @ApiModelProperty("The ID of the Process Group")
     public String getId() {
         return id;
     }
@@ -67,12 +44,7 @@ public class ProcessGroupStatusDTO extends StatusDTO {
         this.id = id;
     }
 
-    /**
-     * @return name of this process group
-     */
-    @ApiModelProperty(
-            value = "The name of this process group."
-    )
+    @ApiModelProperty("The name of the PRocess Group")
     public String getName() {
         return name;
     }
@@ -81,274 +53,23 @@ public class ProcessGroupStatusDTO extends StatusDTO {
         this.name = name;
     }
 
-    /**
-     * @return active thread count for this process group
-     */
-    @ApiModelProperty(
-            value = "The active thread count for this process group."
-    )
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
-
-    /**
-     * The status of all connections in this process group.
-     *
-     * @return The status of all connections
-     */
-    @ApiModelProperty(
-            value = "The status of all conenctions in the process group."
-    )
-    public Collection<ConnectionStatusDTO> getConnectionStatus() {
-        return connectionStatus;
-    }
-
-    public void setConnectionStatus(Collection<ConnectionStatusDTO> connectionStatus) {
-        this.connectionStatus = connectionStatus;
-    }
-
-    /**
-     * The status of all process groups in this process group.
-     *
-     * @return The status of all process groups
-     */
-    @ApiModelProperty(
-            value = "The status of all process groups in the process group."
-    )
-    public Collection<ProcessGroupStatusDTO> getProcessGroupStatus() {
-        return processGroupStatus;
-    }
-
-    public void setProcessGroupStatus(Collection<ProcessGroupStatusDTO> processGroupStatus) {
-        this.processGroupStatus = processGroupStatus;
+    @ApiModelProperty("The aggregate status of all nodes in the cluster")
+    public ProcessGroupStatusSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
     }
 
-    /**
-     * The status of all remote process groups in this process group.
-     *
-     * @return The status of all remote process groups
-     */
-    @ApiModelProperty(
-            value = "The status of all remote process groups in the process group.."
-    )
-    public Collection<RemoteProcessGroupStatusDTO> getRemoteProcessGroupStatus() {
-        return remoteProcessGroupStatus;
+    public void setAggregateSnapshot(ProcessGroupStatusSnapshotDTO aggregateSnapshot) {
+        this.aggregateSnapshot = aggregateSnapshot;
     }
 
-    public void setRemoteProcessGroupStatus(final Collection<RemoteProcessGroupStatusDTO> remoteProcessGroupStatus) {
-        this.remoteProcessGroupStatus = remoteProcessGroupStatus;
+    @ApiModelProperty("The status reported by each node in the cluster. If the NiFi instance is a standalone instance, rather than "
+        + "a clustered instance, this value may be null.")
+    public List<NodeProcessGroupStatusSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
     }
 
-    /**
-     * The status of all processors in this process group.
-     *
-     * @return The status of all processors
-     */
-    @ApiModelProperty(
-            value = "The status of all processors in the process group."
-    )
-    public Collection<ProcessorStatusDTO> getProcessorStatus() {
-        return processorStatus;
-    }
-
-    public void setProcessorStatus(Collection<ProcessorStatusDTO> processorStatus) {
-        this.processorStatus = processorStatus;
-    }
-
-    /**
-     * The status of all input ports in this process group.
-     *
-     * @return The status of all input ports
-     */
-    @ApiModelProperty(
-            value = "The status of all input ports in the process group."
-    )
-    public Collection<PortStatusDTO> getInputPortStatus() {
-        return inputPortStatus;
-    }
-
-    public void setInputPortStatus(Collection<PortStatusDTO> inputPortStatus) {
-        this.inputPortStatus = inputPortStatus;
-    }
-
-    /**
-     * The status of all output ports in this process group.
-     *
-     * @return The status of all output ports
-     */
-    @ApiModelProperty(
-            value = "The status of all output ports in the process group."
-    )
-    public Collection<PortStatusDTO> getOutputPortStatus() {
-        return outputPortStatus;
-    }
-
-    public void setOutputPortStatus(Collection<PortStatusDTO> outputPortStatus) {
-        this.outputPortStatus = outputPortStatus;
-    }
-
-    /**
-     * The output stats for this process group.
-     *
-     * @return The output stats
-     */
-    @ApiModelProperty(
-            value = "The output count/size for the process group in the last 5 minutes."
-    )
-    public String getOutput() {
-        return output;
-    }
-
-    public void setOutput(String output) {
-        this.output = output;
-    }
-
-    /**
-     * 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
-     */
-    @ApiModelProperty(
-            value = "The count/size transferred to/frome queues in the process group in the last 5 minutes."
-    )
-    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
-     */
-    @ApiModelProperty(
-            value = "The count/size sent to the process group in the last 5 minutes."
-    )
-    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
-     */
-    @ApiModelProperty(
-            value = "The count/size sent from this process group in the last 5 minutes."
-    )
-    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
-     */
-    @ApiModelProperty(
-            value = "The count that is queued for the 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
-     */
-    @ApiModelProperty(
-            value = "The size that is queued for the 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
-     */
-    @ApiModelProperty(
-            value = "The count/size that is queued in the the process group."
-    )
-    public String getQueued() {
-        return queued;
-    }
-
-    public void setQueued(String queued) {
-        this.queued = queued;
-    }
-
-    /**
-     * The read stats for this process group.
-     *
-     * @return The read stats
-     */
-    @ApiModelProperty(
-            value = "The number of bytes read in the last 5 minutes."
-    )
-    public String getRead() {
-        return read;
-    }
-
-    public void setRead(String read) {
-        this.read = read;
-    }
-
-    /**
-     * The written stats for this process group.
-     *
-     * @return The written stats
-     */
-    @ApiModelProperty(
-            value = "The number of bytes written in the last 5 minutes."
-    )
-    public String getWritten() {
-        return written;
-    }
-
-    public void setWritten(String written) {
-        this.written = written;
-    }
-
-    /**
-     * The input stats for this process group.
-     *
-     * @return The input stats
-     */
-    @ApiModelProperty(
-            value = "The input count/size for the process group in the last 5 minutes."
-    )
-    public String getInput() {
-        return input;
-    }
-
-    public void setInput(String input) {
-        this.input = input;
+    public void setNodeSnapshots(List<NodeProcessGroupStatusSnapshotDTO> nodeSnapshots) {
+        this.nodeSnapshots = nodeSnapshots;
     }
 
     /**
@@ -358,7 +79,7 @@ public class ProcessGroupStatusDTO extends StatusDTO {
      */
     @XmlJavaTypeAdapter(TimeAdapter.class)
     @ApiModelProperty(
-            value = "The time the status for the process group was last refreshed."
+        value = "The time the status for the process group was last refreshed."
     )
     public Date getStatsLastRefreshed() {
         return statsLastRefreshed;
@@ -367,5 +88,4 @@ public class ProcessGroupStatusDTO extends StatusDTO {
     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/ProcessGroupStatusSnapshotDTO.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/ProcessGroupStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusSnapshotDTO.java
new file mode 100644
index 0000000..e5c61dd
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusSnapshotDTO.java
@@ -0,0 +1,532 @@
+/*
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * The status for a process group in this NiFi.
+ */
+@XmlType(name = "processGroupStatusSnapshot")
+public class ProcessGroupStatusSnapshotDTO implements Cloneable {
+
+    private String id;
+    private String name;
+    private Collection<ConnectionStatusSnapshotDTO> connectionStatus;
+    private Collection<ProcessorStatusSnapshotDTO> processorStatus;
+    private Collection<ProcessGroupStatusSnapshotDTO> processGroupStatus;
+    private Collection<RemoteProcessGroupStatusSnapshotDTO> remoteProcessGroupStatus;
+    private Collection<PortStatusSnapshotDTO> inputPortStatus;
+    private Collection<PortStatusSnapshotDTO> outputPortStatus;
+
+    private Integer flowFilesIn = 0;
+    private Long bytesIn = 0L;
+    private String input;
+
+    private Integer flowFilesQueued = 0;
+    private Long bytesQueued = 0L;
+    private String queued;
+    private String queuedCount;
+    private String queuedSize;
+
+    private Long bytesRead = 0L;
+    private String read;
+    private Long bytesWritten = 0L;
+    private String written;
+
+    private Integer flowFilesOut = 0;
+    private Long bytesOut = 0L;
+    private String output;
+
+    private Integer flowFilesTransferred = 0;
+    private Long bytesTransferred = 0L;
+    private String transferred;
+
+    private Long bytesReceived = 0L;
+    private Integer flowFilesReceived = 0;
+    private String received;
+
+    private Long bytesSent = 0L;
+    private Integer flowFilesSent = 0;
+    private String sent;
+
+    private Integer activeThreadCount = 0;
+
+    /**
+     * The id for the process group.
+     *
+     * @return The id for the process group
+     */
+    @ApiModelProperty("The id of the process group.")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return name of this process group
+     */
+    @ApiModelProperty("The name of this process group.")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return active thread count for this process group
+     */
+    @ApiModelProperty("The active thread count for this process group.")
+    public Integer getActiveThreadCount() {
+        return activeThreadCount;
+    }
+
+    public void setActiveThreadCount(Integer activeThreadCount) {
+        this.activeThreadCount = activeThreadCount;
+    }
+
+    /**
+     * The status of all connections in this process group.
+     *
+     * @return The status of all connections
+     */
+    @ApiModelProperty("The status of all conenctions in the process group.")
+    public Collection<ConnectionStatusSnapshotDTO> getConnectionStatusSnapshots() {
+        return connectionStatus;
+    }
+
+    public void setConnectionStatusSnapshots(Collection<ConnectionStatusSnapshotDTO> connectionStatus) {
+        this.connectionStatus = connectionStatus;
+    }
+
+    /**
+     * The status of all process groups in this process group.
+     *
+     * @return The status of all process groups
+     */
+    @ApiModelProperty("The status of all process groups in the process group.")
+    public Collection<ProcessGroupStatusSnapshotDTO> getProcessGroupStatusSnapshots() {
+        return processGroupStatus;
+    }
+
+    public void setProcessGroupStatusSnapshots(Collection<ProcessGroupStatusSnapshotDTO> processGroupStatus) {
+        this.processGroupStatus = processGroupStatus;
+    }
+
+    /**
+     * The status of all remote process groups in this process group.
+     *
+     * @return The status of all remote process groups
+     */
+    @ApiModelProperty("The status of all remote process groups in the process group.")
+    public Collection<RemoteProcessGroupStatusSnapshotDTO> getRemoteProcessGroupStatusSnapshots() {
+        return remoteProcessGroupStatus;
+    }
+
+    public void setRemoteProcessGroupStatusSnapshots(final Collection<RemoteProcessGroupStatusSnapshotDTO> remoteProcessGroupStatus) {
+        this.remoteProcessGroupStatus = remoteProcessGroupStatus;
+    }
+
+    /**
+     * The status of all processors in this process group.
+     *
+     * @return The status of all processors
+     */
+    @ApiModelProperty("The status of all processors in the process group.")
+    public Collection<ProcessorStatusSnapshotDTO> getProcessorStatusSnapshots() {
+        return processorStatus;
+    }
+
+    public void setProcessorStatusSnapshots(Collection<ProcessorStatusSnapshotDTO> processorStatus) {
+        this.processorStatus = processorStatus;
+    }
+
+    /**
+     * The status of all input ports in this process group.
+     *
+     * @return The status of all input ports
+     */
+    @ApiModelProperty("The status of all input ports in the process group.")
+    public Collection<PortStatusSnapshotDTO> getInputPortStatusSnapshots() {
+        return inputPortStatus;
+    }
+
+    public void setInputPortStatusSnapshots(Collection<PortStatusSnapshotDTO> inputPortStatus) {
+        this.inputPortStatus = inputPortStatus;
+    }
+
+    /**
+     * The status of all output ports in this process group.
+     *
+     * @return The status of all output ports
+     */
+    @ApiModelProperty("The status of all output ports in the process group.")
+    public Collection<PortStatusSnapshotDTO> getOutputPortStatusSnapshots() {
+        return outputPortStatus;
+    }
+
+    public void setOutputPortStatusSnapshots(Collection<PortStatusSnapshotDTO> outputPortStatus) {
+        this.outputPortStatus = outputPortStatus;
+    }
+
+    /**
+     * The output stats for this process group.
+     *
+     * @return The output stats
+     */
+    @ApiModelProperty("The output count/size for the process group in the last 5 minutes.")
+    public String getOutput() {
+        return output;
+    }
+
+    /**
+     * 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
+     */
+    @ApiModelProperty("The count/size transferred to/frome queues in the process group in the last 5 minutes.")
+    public String getTransferred() {
+        return transferred;
+    }
+
+    /**
+     * The received stats for this process group. This represents the count/size of flowfiles received.
+     *
+     * @return The received stats for this process group
+     */
+    @ApiModelProperty("The count/size sent to the process group in the last 5 minutes.")
+    public String getReceived() {
+        return received;
+    }
+
+
+    /**
+     * The sent stats for this process group. This represents the count/size of flowfiles sent.
+     *
+     * @return The sent stats for this process group
+     */
+    @ApiModelProperty("The count/size sent from this process group in the last 5 minutes.")
+    public String getSent() {
+        return sent;
+    }
+
+
+    /**
+     * The queued count for this process group.
+     *
+     * @return The queued count for this process group
+     */
+    @ApiModelProperty("The count that is queued for the process group.")
+    public String getQueuedCount() {
+        return queuedCount;
+    }
+
+
+    /**
+     * The queued size for this process group.
+     *
+     * @return The queued size for this process group
+     */
+    @ApiModelProperty("The size that is queued for the process group.")
+    public String getQueuedSize() {
+        return queuedSize;
+    }
+
+
+    /**
+     * The queued stats for this process group.
+     *
+     * @return The queued stats
+     */
+    @ApiModelProperty("The count/size that is queued in the the process group.")
+    public String getQueued() {
+        return queued;
+    }
+
+
+    /**
+     * The read stats for this process group.
+     *
+     * @return The read stats
+     */
+    @ApiModelProperty("The number of bytes read in the last 5 minutes.")
+    public String getRead() {
+        return read;
+    }
+
+
+    /**
+     * The written stats for this process group.
+     *
+     * @return The written stats
+     */
+    @ApiModelProperty("The number of bytes written in the last 5 minutes.")
+    public String getWritten() {
+        return written;
+    }
+
+
+    /**
+     * The input stats for this process group.
+     *
+     * @return The input stats
+     */
+    @ApiModelProperty("The input count/size for the process group in the last 5 minutes (pretty printed).")
+    public String getInput() {
+        return input;
+    }
+
+
+    @ApiModelProperty("The number of FlowFiles that have come into this ProcessGroup in the last 5 minutes")
+    public Integer getFlowFilesIn() {
+        return flowFilesIn;
+    }
+
+    public void setFlowFilesIn(Integer flowFilesIn) {
+        this.flowFilesIn = flowFilesIn;
+    }
+
+    @ApiModelProperty("The number of bytes that have come into this ProcessGroup in the last 5 minutes")
+    public Long getBytesIn() {
+        return bytesIn;
+    }
+
+    public void setBytesIn(Long bytesIn) {
+        this.bytesIn = bytesIn;
+    }
+
+    @ApiModelProperty("The number of FlowFiles that are queued up in this ProcessGroup right now")
+    public Integer getFlowFilesQueued() {
+        return flowFilesQueued;
+    }
+
+    public void setFlowFilesQueued(Integer flowFilesQueued) {
+        this.flowFilesQueued = flowFilesQueued;
+    }
+
+    @ApiModelProperty("The number of bytes that are queued up in this ProcessGroup right now")
+    public Long getBytesQueued() {
+        return bytesQueued;
+    }
+
+    public void setBytesQueued(Long bytesQueued) {
+        this.bytesQueued = bytesQueued;
+    }
+
+    @ApiModelProperty("The number of bytes read by components in this ProcessGroup in the last 5 minutes")
+    public Long getBytesRead() {
+        return bytesRead;
+    }
+
+    public void setBytesRead(Long bytesRead) {
+        this.bytesRead = bytesRead;
+    }
+
+    @ApiModelProperty("The number of bytes written by components in this ProcessGroup in the last 5 minutes")
+    public Long getBytesWritten() {
+        return bytesWritten;
+    }
+
+    public void setBytesWritten(Long bytesWritten) {
+        this.bytesWritten = bytesWritten;
+    }
+
+    @ApiModelProperty("The number of FlowFiles transferred out of this ProcessGroup in the last 5 minutes")
+    public Integer getFlowFilesOut() {
+        return flowFilesOut;
+    }
+
+    public void setFlowFilesOut(Integer flowFilesOut) {
+        this.flowFilesOut = flowFilesOut;
+    }
+
+    @ApiModelProperty("The number of bytes transferred out of this ProcessGroup in the last 5 minutes")
+    public Long getBytesOut() {
+        return bytesOut;
+    }
+
+    public void setBytesOut(Long bytesOut) {
+        this.bytesOut = bytesOut;
+    }
+
+    @ApiModelProperty("The number of FlowFiles transferred in this ProcessGroup in the last 5 minutes")
+    public Integer getFlowFilesTransferred() {
+        return flowFilesTransferred;
+    }
+
+    public void setFlowFilesTransferred(Integer flowFilesTransferred) {
+        this.flowFilesTransferred = flowFilesTransferred;
+    }
+
+    @ApiModelProperty("The number of bytes transferred in this ProcessGroup in the last 5 minutes")
+    public Long getBytesTransferred() {
+        return bytesTransferred;
+    }
+
+    public void setBytesTransferred(Long bytesTransferred) {
+        this.bytesTransferred = bytesTransferred;
+    }
+
+    @ApiModelProperty("The number of bytes received from external sources by components within this ProcessGroup in the last 5 minutes")
+    public Long getBytesReceived() {
+        return bytesReceived;
+    }
+
+    public void setBytesReceived(Long bytesReceived) {
+        this.bytesReceived = bytesReceived;
+    }
+
+    @ApiModelProperty("The number of bytes sent to an external sink by components within this ProcessGroup in the last 5 minutes")
+    public Long getBytesSent() {
+        return bytesSent;
+    }
+
+    public void setBytesSent(Long bytesSent) {
+        this.bytesSent = bytesSent;
+    }
+
+    @ApiModelProperty("The number of FlowFiles sent to an external sink by components within this ProcessGroup in the last 5 minutes")
+    public Integer getFlowFilesSent() {
+        return flowFilesSent;
+    }
+
+    public void setFlowFilesSent(Integer flowFilesSent) {
+        this.flowFilesSent = flowFilesSent;
+    }
+
+    @ApiModelProperty("The number of FlowFiles received from external sources by components within this ProcessGroup in the last 5 minutes")
+    public Integer getFlowFilesReceived() {
+        return flowFilesReceived;
+    }
+
+    public void setFlowFilesReceived(Integer flowFilesReceived) {
+        this.flowFilesReceived = flowFilesReceived;
+    }
+
+    public void setInput(String input) {
+        this.input = input;
+    }
+
+    public void setQueued(String queued) {
+        this.queued = queued;
+    }
+
+    public void setQueuedCount(String queuedCount) {
+        this.queuedCount = queuedCount;
+    }
+
+    public void setQueuedSize(String queuedSize) {
+        this.queuedSize = queuedSize;
+    }
+
+    public void setRead(String read) {
+        this.read = read;
+    }
+
+    public void setWritten(String written) {
+        this.written = written;
+    }
+
+    public void setOutput(String output) {
+        this.output = output;
+    }
+
+    public void setTransferred(String transferred) {
+        this.transferred = transferred;
+    }
+
+    public void setReceived(String received) {
+        this.received = received;
+    }
+
+    public void setSent(String sent) {
+        this.sent = sent;
+    }
+
+    @Override
+    public ProcessGroupStatusSnapshotDTO clone() {
+        final ProcessGroupStatusSnapshotDTO other = new ProcessGroupStatusSnapshotDTO();
+        other.setId(getId());
+        other.setName(getName());
+
+        other.setBytesIn(getBytesIn());
+        other.setFlowFilesIn(getFlowFilesIn());
+        other.setInput(getInput());
+
+        other.setBytesQueued(getBytesQueued());
+        other.setFlowFilesQueued(getFlowFilesQueued());
+        other.setQueued(getQueued());
+        other.setQueuedCount(getQueuedCount());
+        other.setQueuedSize(getQueuedSize());
+
+        other.setBytesRead(getBytesRead());
+        other.setRead(getRead());
+        other.setBytesWritten(getBytesWritten());
+        other.setWritten(getWritten());
+
+        other.setBytesOut(getBytesOut());
+        other.setFlowFilesOut(getFlowFilesOut());
+        other.setOutput(getOutput());
+
+        other.setBytesTransferred(getBytesTransferred());
+        other.setFlowFilesTransferred(getFlowFilesTransferred());
+        other.setTransferred(getTransferred());
+
+        other.setBytesReceived(getBytesReceived());
+        other.setFlowFilesReceived(getFlowFilesReceived());
+        other.setReceived(getReceived());
+        other.setBytesSent(getBytesSent());
+        other.setFlowFilesSent(getFlowFilesSent());
+        other.setSent(getSent());
+
+        other.setActiveThreadCount(getActiveThreadCount());
+
+        other.setConnectionStatusSnapshots(copy(getConnectionStatusSnapshots()));
+        other.setProcessorStatusSnapshots(copy(getProcessorStatusSnapshots()));
+        other.setRemoteProcessGroupStatusSnapshots(copy(getRemoteProcessGroupStatusSnapshots()));
+        other.setInputPortStatusSnapshots(copy(getInputPortStatusSnapshots()));
+        other.setOutputPortStatusSnapshots(copy(getOutputPortStatusSnapshots()));
+
+        if (processGroupStatus != null) {
+            final List<ProcessGroupStatusSnapshotDTO> childGroups = new ArrayList<>();
+            for (final ProcessGroupStatusSnapshotDTO procGroupStatus : processGroupStatus) {
+                childGroups.add(procGroupStatus.clone());
+            }
+            other.setProcessGroupStatusSnapshots(childGroups);
+        }
+
+        return other;
+    }
+
+    private <T> Collection<T> copy(final Collection<T> original) {
+        if (original == null) {
+            return null;
+        }
+
+        return new ArrayList<T>(original);
+    }
+}

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/ProcessorStatusDTO.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/ProcessorStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java
index 1899418..a4729c4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java
@@ -14,40 +14,44 @@
  * 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;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
 
 /**
  * DTO for serializing the status of a processor.
  */
 @XmlType(name = "processorStatus")
-public class ProcessorStatusDTO extends StatusDTO {
-
-    private String id;
+public class ProcessorStatusDTO implements Cloneable {
     private String groupId;
+    private String id;
     private String name;
     private String type;
     private String runStatus;
+    private Date statsLastRefreshed;
 
-    private String read;
-    private String written;
+    private ProcessorStatusSnapshotDTO aggregateSnapshot;
+    private List<NodeProcessorStatusSnapshotDTO> nodeSnapshots;
 
-    private String input;
-    private String output;
+    @ApiModelProperty("The unique ID of the process group that the Processor belongs to")
+    public String getGroupId() {
+        return groupId;
+    }
 
-    private String tasks;
-    private String tasksDuration;
-    private Integer activeThreadCount;
+    public void setGroupId(String groupId) {
+        this.groupId = groupId;
+    }
 
-    /* getters / setters */
-    /**
-     * @return The processor id
-     */
-    @ApiModelProperty(
-            value = "The id of the processor."
-    )
+    @ApiModelProperty("The unique ID of the Processor")
     public String getId() {
         return id;
     }
@@ -56,12 +60,7 @@ public class ProcessorStatusDTO extends StatusDTO {
         this.id = id;
     }
 
-    /**
-     * @return The processor name
-     */
-    @ApiModelProperty(
-            value = "The name of the prcessor."
-    )
+    @ApiModelProperty("The name of the Processor")
     public String getName() {
         return name;
     }
@@ -70,12 +69,7 @@ public class ProcessorStatusDTO extends StatusDTO {
         this.name = name;
     }
 
-    /**
-     * @return The processor type
-     */
-    @ApiModelProperty(
-            value = "The type of the processor."
-    )
+    @ApiModelProperty("The type of the Processor")
     public String getType() {
         return type;
     }
@@ -84,13 +78,7 @@ public class ProcessorStatusDTO extends StatusDTO {
         this.type = type;
     }
 
-    /**
-     * @return run status of this processor
-     */
-    @ApiModelProperty(
-            value = "The state of the processor.",
-            allowableValues = "RUNNING, STOPPED, DISABLED, INVALID"
-    )
+    @ApiModelProperty("The run status of the Processor")
     public String getRunStatus() {
         return runStatus;
     }
@@ -99,116 +87,54 @@ public class ProcessorStatusDTO 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;
+    @XmlJavaTypeAdapter(TimeAdapter.class)
+    @ApiModelProperty("The timestamp of when the stats were last refreshed")
+    public Date getStatsLastRefreshed() {
+        return statsLastRefreshed;
     }
 
-    public void setInput(String input) {
-        this.input = input;
+    public void setStatsLastRefreshed(Date statsLastRefreshed) {
+        this.statsLastRefreshed = statsLastRefreshed;
     }
 
-    /**
-     * @return number of bytes read
-     */
-    @ApiModelProperty(
-            value = "The number of bytes read in the last 5 minutes."
-    )
-    public String getRead() {
-        return read;
+    @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 ProcessorStatusSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
     }
 
-    public void setRead(String read) {
-        this.read = read;
+    public void setAggregateSnapshot(ProcessorStatusSnapshotDTO aggregateSnapshot) {
+        this.aggregateSnapshot = aggregateSnapshot;
     }
 
-    /**
-     * @return number of bytes written
-     */
-    @ApiModelProperty(
-            value = "The number of bytes written in the last 5 minutes."
-    )
-    public String getWritten() {
-        return written;
+    @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<NodeProcessorStatusSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
     }
 
-    public void setWritten(String written) {
-        this.written = written;
+    public void setNodeSnapshots(List<NodeProcessorStatusSnapshotDTO> nodeSnapshots) {
+        this.nodeSnapshots = nodeSnapshots;
     }
 
-    /**
-     * @return the ID of the Process Group to which this processor belongs.
-     */
-    @ApiModelProperty(
-            value = "The id of the parent process group to which the processor belongs."
-    )
-    public String getGroupId() {
-        return groupId;
-    }
+    @Override
+    public ProcessorStatusDTO clone() {
+        final ProcessorStatusDTO other = new ProcessorStatusDTO();
+        other.setGroupId(getGroupId());
+        other.setId(getId());
+        other.setName(getName());
+        other.setRunStatus(getRunStatus());
+        other.setType(getType());
+        other.setStatsLastRefreshed(getStatsLastRefreshed());
+        other.setAggregateSnapshot(getAggregateSnapshot().clone());
 
-    public void setGroupId(final String groupId) {
-        this.groupId = groupId;
-    }
-
-    /**
-     * @return The total count and size of flow files that have been processed in the last five minutes
-     */
-    @ApiModelProperty(
-            value = "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;
-    }
-
-    /**
-     * @return number of threads currently running for this Processor
-     */
-    @ApiModelProperty(
-            value = "The number of threads currently executing in the processor."
-    )
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer threadCount) {
-        this.activeThreadCount = threadCount;
-    }
-
-    /**
-     * @return number of task this connectable has had over the last 5 minutes
-     */
-    @ApiModelProperty(
-            value = "The total number of task this connectable has completed over the last 5 minutes."
-    )
-    public String getTasks() {
-        return tasks;
-    }
-
-    public void setTasks(String tasks) {
-        this.tasks = tasks;
-    }
-
-    /**
-     * @return total duration of all tasks for this connectable over the last 5 minutes
-     */
-    @ApiModelProperty(
-            value = "The total duration of all tasks for this connectable over the last 5 minutes."
-    )
-    public String getTasksDuration() {
-        return tasksDuration;
-    }
+        final List<NodeProcessorStatusSnapshotDTO> nodeStatuses = getNodeSnapshots();
+        final List<NodeProcessorStatusSnapshotDTO> nodeStatusClones = new ArrayList<>(nodeStatuses.size());
+        for (final NodeProcessorStatusSnapshotDTO status : nodeStatuses) {
+            nodeStatusClones.add(status.clone());
+        }
+        other.setNodeSnapshots(nodeStatusClones);
 
-    public void setTasksDuration(String tasksDuration) {
-        this.tasksDuration = tasksDuration;
+        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/ProcessorStatusSnapshotDTO.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/ProcessorStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusSnapshotDTO.java
new file mode 100644
index 0000000..032fb57
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusSnapshotDTO.java
@@ -0,0 +1,301 @@
+/*
+ * 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 status of a processor.
+ */
+@XmlType(name = "processorStatusSnapshot")
+public class ProcessorStatusSnapshotDTO implements Cloneable {
+
+    private String id;
+    private String groupId;
+    private String name;
+    private String type;
+    private String runStatus;
+
+    private Long bytesRead = 0L;
+    private Long bytesWritten = 0L;
+    private String read;
+    private String written;
+
+    private Integer flowFilesIn = 0;
+    private Long bytesIn = 0L;
+    private String input;
+
+    private Integer flowFilesOut = 0;
+    private Long bytesOut = 0L;
+    private String output;
+
+    private Integer taskCount = 0;
+    private Long tasksDurationNanos = 0L;
+    private String tasks;
+    private String tasksDuration;
+    private Integer activeThreadCount = 0;
+
+    /* getters / setters */
+    /**
+     * @return The processor id
+     */
+    @ApiModelProperty("The id of the processor.")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return The processor name
+     */
+    @ApiModelProperty("The name of the prcessor.")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return The processor type
+     */
+    @ApiModelProperty("The type of the processor.")
+    public String getType() {
+        return type;
+    }
+
+    public void setType(String type) {
+        this.type = type;
+    }
+
+    /**
+     * @return run status of this processor
+     */
+    @ApiModelProperty(
+            value = "The state of the processor.",
+            allowableValues = "RUNNING, STOPPED, DISABLED, INVALID"
+    )
+    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 number of bytes read
+     */
+    @ApiModelProperty("The number of bytes read in the last 5 minutes.")
+    public String getRead() {
+        return read;
+    }
+
+    public void setRead(String read) {
+        this.read = read;
+    }
+
+    /**
+     * @return number of bytes written
+     */
+    @ApiModelProperty("The number of bytes written in the last 5 minutes.")
+    public String getWritten() {
+        return written;
+    }
+
+    public void setWritten(String written) {
+        this.written = written;
+    }
+
+    /**
+     * @return the ID of the Process Group to which this processor belongs.
+     */
+    @ApiModelProperty("The id of the parent process group to which the processor belongs.")
+    public String getGroupId() {
+        return groupId;
+    }
+
+    public void setGroupId(final String groupId) {
+        this.groupId = groupId;
+    }
+
+    /**
+     * @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;
+    }
+
+    /**
+     * @return number of threads currently running for this Processor
+     */
+    @ApiModelProperty("The number of threads currently executing in the processor.")
+    public Integer getActiveThreadCount() {
+        return activeThreadCount;
+    }
+
+    public void setActiveThreadCount(Integer threadCount) {
+        this.activeThreadCount = threadCount;
+    }
+
+    /**
+     * @return number of task this connectable has had over the last 5 minutes
+     */
+    @ApiModelProperty("The total number of task this connectable has completed over the last 5 minutes.")
+    public String getTasks() {
+        return tasks;
+    }
+
+    public void setTasks(String tasks) {
+        this.tasks = tasks;
+    }
+
+    /**
+     * @return total duration of all tasks for this connectable over the last 5 minutes
+     */
+    @ApiModelProperty("The total duration of all tasks for this connectable over the last 5 minutes.")
+    public String getTasksDuration() {
+        return tasksDuration;
+    }
+
+    public void setTasksDuration(String tasksDuration) {
+        this.tasksDuration = tasksDuration;
+    }
+
+    @ApiModelProperty("The number of bytes read by this Processor in the last 5 mintues")
+    public Long getBytesRead() {
+        return bytesRead;
+    }
+
+    public void setBytesRead(Long bytesRead) {
+        this.bytesRead = bytesRead;
+    }
+
+    @ApiModelProperty("The number of bytes written by this Processor in the last 5 minutes")
+    public Long getBytesWritten() {
+        return bytesWritten;
+    }
+
+    public void setBytesWritten(Long bytesWritten) {
+        this.bytesWritten = bytesWritten;
+    }
+
+    @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 the 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 transferred to a Connection in the last 5 minutes")
+    public Integer getFlowFilesOut() {
+        return flowFilesOut;
+    }
+
+    public void setFlowFilesOut(Integer flowFilesOut) {
+        this.flowFilesOut = flowFilesOut;
+    }
+
+    @ApiModelProperty("The size of the FlowFiles transferred to a Connection in the last 5 minutes")
+    public Long getBytesOut() {
+        return bytesOut;
+    }
+
+    public void setBytesOut(Long bytesOut) {
+        this.bytesOut = bytesOut;
+    }
+
+    @ApiModelProperty("The number of times this Processor has run in the last 5 minutes")
+    public Integer getTaskCount() {
+        return taskCount;
+    }
+
+    public void setTaskCount(Integer taskCount) {
+        this.taskCount = taskCount;
+    }
+
+    @ApiModelProperty("The number of nanoseconds that this Processor has spent running in the last 5 minutes")
+    public Long getTasksDurationNanos() {
+        return tasksDurationNanos;
+    }
+
+    public void setTasksDurationNanos(Long taskNanos) {
+        this.tasksDurationNanos = taskNanos;
+    }
+
+    @Override
+    public ProcessorStatusSnapshotDTO clone() {
+        final ProcessorStatusSnapshotDTO other = new ProcessorStatusSnapshotDTO();
+        other.setId(getId());
+        other.setGroupId(getGroupId());
+        other.setName(getName());
+        other.setType(getType());
+
+        other.setRunStatus(getRunStatus());
+        other.setBytesRead(getBytesRead());
+        other.setBytesWritten(getBytesWritten());
+        other.setFlowFilesIn(getFlowFilesIn());
+        other.setBytesIn(getBytesIn());
+        other.setFlowFilesOut(getFlowFilesOut());
+        other.setBytesOut(getBytesOut());
+        other.setTaskCount(getTaskCount());
+        other.setTasksDuration(getTasksDuration());
+        other.setTasksDurationNanos(getTasksDurationNanos());
+        other.setActiveThreadCount(getActiveThreadCount());
+        other.setInput(getInput());
+        other.setOutput(getOutput());
+        other.setRead(getRead());
+        other.setWritten(getWritten());
+        other.setTasks(getTasks());
+
+        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/RemotePortStatusDTO.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/RemotePortStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java
index 6778b0d..a6723c7 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java
@@ -23,7 +23,7 @@ import javax.xml.bind.annotation.XmlType;
  * The status of a Port on a remote NiFi instance.
  */
 @XmlType(name = "remotePortStatus")
-public class RemotePortStatusDTO {
+public class RemotePortStatusDTO implements Cloneable {
 
     private String id;
     private String connectionId;
@@ -101,4 +101,14 @@ public class RemotePortStatusDTO {
         this.running = running;
     }
 
+    @Override
+    public RemotePortStatusDTO clone() {
+        final RemotePortStatusDTO other = new RemotePortStatusDTO();
+        other.setId(getId());
+        other.setName(getName());
+        other.setConnectionId(getConnectionId());
+        other.setExists(getExists());
+        other.setRunning(getRunning());
+        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/RemoteProcessGroupStatusDTO.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/RemoteProcessGroupStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java
index f556deb..62685b1 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java
@@ -14,50 +14,33 @@
  * 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 of a remote process group in this NiFi.
- */
-@XmlType(name = "remoteProcessGroupStatus")
-public class RemoteProcessGroupStatusDTO extends StatusDTO {
+import com.wordnik.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
 
-    private String id;
+@XmlType(name = "remoteProcessGroupStatus")
+public class RemoteProcessGroupStatusDTO {
     private String groupId;
+    private String id;
     private String name;
     private String targetUri;
     private String transmissionStatus;
-    private Integer activeThreadCount;
+    private Date statsLastRefreshed;
 
     private List<String> authorizationIssues;
 
-    private String sent;
-    private String received;
+    private RemoteProcessGroupStatusSnapshotDTO aggregateSnapshot;
+    private List<NodeRemoteProcessGroupStatusSnapshotDTO> nodeSnapshots;
 
-    /**
-     * @return The id for the remote process group
-     */
-    @ApiModelProperty(
-            value = "The id of the remote process group."
-    )
-    public String getId() {
-        return id;
-    }
-
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    /**
-     * @return id of the group this remote process group is in
-     */
-    @ApiModelProperty(
-            value = "The id of the parent process group the remote process group resides in."
-    )
+    @ApiModelProperty("The unique ID of the process group that the Processor belongs to")
     public String getGroupId() {
         return groupId;
     }
@@ -66,26 +49,16 @@ public class RemoteProcessGroupStatusDTO extends StatusDTO {
         this.groupId = groupId;
     }
 
-    /**
-     * @return URI of the target system
-     */
-    @ApiModelProperty(
-            value = "The URI of the target system."
-    )
-    public String getTargetUri() {
-        return targetUri;
+    @ApiModelProperty("The unique ID of the Processor")
+    public String getId() {
+        return id;
     }
 
-    public void setTargetUri(String targetUri) {
-        this.targetUri = targetUri;
+    public void setId(String id) {
+        this.id = id;
     }
 
-    /**
-     * @return name of this remote process group
-     */
-    @ApiModelProperty(
-            value = "The name of the remote process group."
-    )
+    @ApiModelProperty("The name of the remote process group.")
     public String getName() {
         return name;
     }
@@ -94,12 +67,7 @@ public class RemoteProcessGroupStatusDTO extends StatusDTO {
         this.name = name;
     }
 
-    /**
-     * @return transmission status of this remote process group
-     */
-    @ApiModelProperty(
-            value = "The transmission status of the remote process group."
-    )
+    @ApiModelProperty("The transmission status of the remote process group.")
     public String getTransmissionStatus() {
         return transmissionStatus;
     }
@@ -108,26 +76,8 @@ public class RemoteProcessGroupStatusDTO extends StatusDTO {
         this.transmissionStatus = transmissionStatus;
     }
 
-    /**
-     * @return number of active threads
-     */
-    @ApiModelProperty(
-            value = "The number of active threads for the remote process group."
-    )
-    public Integer getActiveThreadCount() {
-        return activeThreadCount;
-    }
-
-    public void setActiveThreadCount(Integer activeThreadCount) {
-        this.activeThreadCount = activeThreadCount;
-    }
 
-    /**
-     * @return any remote authorization issues for this remote process group
-     */
-    @ApiModelProperty(
-            value = "Any remote authorization issues for the remote process group."
-    )
+    @ApiModelProperty("Any remote authorization issues for the remote process group.")
     public List<String> getAuthorizationIssues() {
         return authorizationIssues;
     }
@@ -136,32 +86,49 @@ public class RemoteProcessGroupStatusDTO extends StatusDTO {
         this.authorizationIssues = authorizationIssues;
     }
 
-    /**
-     * @return Formatted description of the amount of data sent to this remote process group
-     */
-    @ApiModelProperty(
-            value = "The count/size of the flowfiles sent to the remote process group in the last 5 minutes."
-    )
-    public String getSent() {
-        return sent;
+    @ApiModelProperty("The URI of the target system.")
+    public String getTargetUri() {
+        return targetUri;
+    }
+
+    public void setTargetUri(String targetUri) {
+        this.targetUri = targetUri;
     }
 
-    public void setSent(String sent) {
-        this.sent = sent;
+    @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 RemoteProcessGroupStatusSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
+    }
+
+    public void setAggregateSnapshot(RemoteProcessGroupStatusSnapshotDTO 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<NodeRemoteProcessGroupStatusSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
+    }
+
+    public void setNodeSnapshots(List<NodeRemoteProcessGroupStatusSnapshotDTO> nodeSnapshots) {
+        this.nodeSnapshots = nodeSnapshots;
     }
 
     /**
-     * @return Formatted description of the amount of data received from this remote process group
+     * When the status for this process group was calculated.
+     *
+     * @return The the status was calculated
      */
+    @XmlJavaTypeAdapter(TimeAdapter.class)
     @ApiModelProperty(
-            value = "The count/size of the flowfiles received from the remote process group in the last 5 minutes."
+        value = "The time the status for the process group was last refreshed."
     )
-    public String getReceived() {
-        return received;
+    public Date getStatsLastRefreshed() {
+        return statsLastRefreshed;
     }
 
-    public void setReceived(String received) {
-        this.received = received;
+    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/RemoteProcessGroupStatusSnapshotDTO.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/RemoteProcessGroupStatusSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusSnapshotDTO.java
new file mode 100644
index 0000000..f0f0d7e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusSnapshotDTO.java
@@ -0,0 +1,215 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+/**
+ * The status of a remote process group in this NiFi.
+ */
+@XmlType(name = "remoteProcessGroupStatusSnapshot")
+public class RemoteProcessGroupStatusSnapshotDTO implements Cloneable {
+
+    private String id;
+    private String groupId;
+    private String name;
+    private String targetUri;
+    private String transmissionStatus;
+    private Integer activeThreadCount;
+
+    private List<String> authorizationIssues;
+
+    private Integer flowFilesSent = 0;
+    private Long bytesSent = 0L;
+    private String sent;
+
+    private Integer flowFilesReceived = 0;
+    private Long bytesReceived = 0L;
+    private String received;
+
+    /**
+     * @return The id for the remote process group
+     */
+    @ApiModelProperty("The id of the remote process group.")
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * @return id of the group this remote process group is in
+     */
+    @ApiModelProperty("The id of the parent process group the remote process group resides in.")
+    public String getGroupId() {
+        return groupId;
+    }
+
+    public void setGroupId(String groupId) {
+        this.groupId = groupId;
+    }
+
+    /**
+     * @return URI of the target system
+     */
+    @ApiModelProperty("The URI of the target system.")
+    public String getTargetUri() {
+        return targetUri;
+    }
+
+    public void setTargetUri(String targetUri) {
+        this.targetUri = targetUri;
+    }
+
+    /**
+     * @return name of this remote process group
+     */
+    @ApiModelProperty("The name of the remote process group.")
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * @return transmission status of this remote process group
+     */
+    @ApiModelProperty("The transmission status of the remote process group.")
+    public String getTransmissionStatus() {
+        return transmissionStatus;
+    }
+
+    public void setTransmissionStatus(String transmissionStatus) {
+        this.transmissionStatus = transmissionStatus;
+    }
+
+    /**
+     * @return number of active threads
+     */
+    @ApiModelProperty("The number of active threads for the remote process group.")
+    public Integer getActiveThreadCount() {
+        return activeThreadCount;
+    }
+
+    public void setActiveThreadCount(Integer activeThreadCount) {
+        this.activeThreadCount = activeThreadCount;
+    }
+
+    /**
+     * @return any remote authorization issues for this remote process group
+     */
+    @ApiModelProperty("Any remote authorization issues for the remote process group.")
+    public List<String> getAuthorizationIssues() {
+        return authorizationIssues;
+    }
+
+    public void setAuthorizationIssues(List<String> authorizationIssues) {
+        this.authorizationIssues = authorizationIssues;
+    }
+
+    /**
+     * @return Formatted description of the amount of data sent to this remote process group
+     */
+    @ApiModelProperty("The count/size of the flowfiles sent to the remote process group in the last 5 minutes.")
+    public String getSent() {
+        return sent;
+    }
+
+    public void setSent(String sent) {
+        this.sent = sent;
+    }
+
+
+    /**
+     * @return Formatted description of the amount of data received from this remote process group
+     */
+    @ApiModelProperty("The count/size of the flowfiles received from the remote process group in the last 5 minutes.")
+    public String getReceived() {
+        return received;
+    }
+
+    public void setReceived(String received) {
+        this.received = received;
+    }
+
+    @ApiModelProperty("The number of FlowFiles sent to the remote process group in the last 5 minutes.")
+    public Integer getFlowFilesSent() {
+        return flowFilesSent;
+    }
+
+    public void setFlowFilesSent(Integer flowFilesSent) {
+        this.flowFilesSent = flowFilesSent;
+    }
+
+    @ApiModelProperty("The size of the FlowFiles sent to the remote process group in the last 5 minutes.")
+    public Long getBytesSent() {
+        return bytesSent;
+    }
+
+    public void setBytesSent(Long bytesSent) {
+        this.bytesSent = bytesSent;
+    }
+
+    @ApiModelProperty("The number of FlowFiles received from the remote process group in the last 5 minutes.")
+    public Integer getFlowFilesReceived() {
+        return flowFilesReceived;
+    }
+
+    public void setFlowFilesReceived(Integer flowFilesReceived) {
+        this.flowFilesReceived = flowFilesReceived;
+    }
+
+    @ApiModelProperty("The size of the FlowFiles received from the remote process group in the last 5 minutes.")
+    public Long getBytesReceived() {
+        return bytesReceived;
+    }
+
+    public void setBytesReceived(Long bytesReceived) {
+        this.bytesReceived = bytesReceived;
+    }
+
+
+    @Override
+    public RemoteProcessGroupStatusSnapshotDTO clone() {
+        final RemoteProcessGroupStatusSnapshotDTO other = new RemoteProcessGroupStatusSnapshotDTO();
+        other.setId(getId());
+        other.setGroupId(getGroupId());
+        other.setName(getName());
+        other.setTargetUri(getTargetUri());
+        other.setTransmissionStatus(getTransmissionStatus());
+        other.setActiveThreadCount(getActiveThreadCount());
+        other.setAuthorizationIssues(getAuthorizationIssues() == null ? null : new ArrayList<String>(getAuthorizationIssues()));
+        other.setFlowFilesSent(getFlowFilesSent());
+        other.setBytesSent(getBytesSent());
+        other.setFlowFilesReceived(getFlowFilesReceived());
+        other.setBytesReceived(getBytesReceived());
+        other.setReceived(getReceived());
+        other.setSent(getSent());
+
+        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/StatusDTO.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/StatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java
deleted file mode 100644
index 39b9c06..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java
+++ /dev/null
@@ -1,45 +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 java.util.List;
-import javax.xml.bind.annotation.XmlType;
-import org.apache.nifi.web.api.dto.BulletinDTO;
-
-/**
- * The status of a component in this NiFi.
- */
-@XmlType(name = "status")
-public abstract class StatusDTO {
-
-    private List<BulletinDTO> bulletins;
-
-    /**
-     * @return Bulletins for this component
-     */
-    @ApiModelProperty(
-            value = "The current bulletins for the component."
-    )
-    public List<BulletinDTO> getBulletins() {
-        return bulletins;
-    }
-
-    public void setBulletins(List<BulletinDTO> bulletins) {
-        this.bulletins = bulletins;
-    }
-}

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/StatusHistoryDTO.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/StatusHistoryDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java
index c0ef33b..3654e10 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java
@@ -17,33 +17,32 @@
 package org.apache.nifi.web.api.dto.status;
 
 import com.wordnik.swagger.annotations.ApiModelProperty;
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
+
+import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
 import java.util.Date;
 import java.util.LinkedHashMap;
 import java.util.List;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
 
 /**
- * History status for a component in this NiFi.
+ * DTO for serializing the status history of a single component across the cluster.
  */
 @XmlType(name = "statusHistory")
 public class StatusHistoryDTO {
 
     private Date generated;
 
-    private LinkedHashMap<String, String> details;
-
+    private LinkedHashMap<String, String> componentDetails;
     private List<StatusDescriptorDTO> fieldDescriptors;
-    private List<StatusSnapshotDTO> statusSnapshots;
+    private List<StatusSnapshotDTO> aggregateSnapshots;
+    private List<NodeStatusSnapshotsDTO> nodeSnapshots;
 
     /**
      * @return when this status history was generated
      */
     @XmlJavaTypeAdapter(TimeAdapter.class)
-    @ApiModelProperty(
-            value = "The timestamp when the status history was generated."
-    )
+    @ApiModelProperty("When the status history was generated.")
     public Date getGenerated() {
         return generated;
     }
@@ -53,25 +52,18 @@ public class StatusHistoryDTO {
     }
 
     /**
-     * @return The component details for this status history
+     * @return key/value pairs that describe the component that the status history belongs to
      */
-    @ApiModelProperty(
-            value = "The component details for the status history."
-    )
-    public LinkedHashMap<String, String> getDetails() {
-        return details;
+    @ApiModelProperty("A Map of key/value pairs that describe the component that the status history belongs to")
+    public LinkedHashMap<String, String> getComponentDetails() {
+        return componentDetails;
     }
 
-    public void setDetails(LinkedHashMap<String, String> details) {
-        this.details = details;
+    public void setComponentDetails(LinkedHashMap<String, String> componentDetails) {
+        this.componentDetails = componentDetails;
     }
 
-    /**
-     * @return Descriptors for each supported status field
-     */
-    @ApiModelProperty(
-            value = "The descriptor for each support status field."
-    )
+    @ApiModelProperty("The Descriptors that provide information on each of the metrics provided in the status history")
     public List<StatusDescriptorDTO> getFieldDescriptors() {
         return fieldDescriptors;
     }
@@ -80,18 +72,24 @@ public class StatusHistoryDTO {
         this.fieldDescriptors = fieldDescriptors;
     }
 
-    /**
-     * @return The status snapshots
-     */
-    @ApiModelProperty(
-            value = "The status snapshots."
-    )
-    public List<StatusSnapshotDTO> getStatusSnapshots() {
-        return statusSnapshots;
+    @ApiModelProperty("A list of StatusSnapshotDTO objects that provide the actual metric values for the component. If the NiFi instance "
+        + "is clustered, this will represent the aggregate status across all nodes. If the NiFi instance is not clustered, this will represent "
+        + "the status of the entire NiFi instance.")
+    public List<StatusSnapshotDTO> getAggregateSnapshots() {
+        return aggregateSnapshots;
     }
 
-    public void setStatusSnapshots(List<StatusSnapshotDTO> statusSnapshots) {
-        this.statusSnapshots = statusSnapshots;
+    public void setAggregateSnapshots(List<StatusSnapshotDTO> aggregateSnapshots) {
+        this.aggregateSnapshots = aggregateSnapshots;
     }
 
+    @ApiModelProperty("The NodeStatusSnapshotsDTO objects that provide the actual metric values for the component, for each node. "
+        + "If the NiFi instance is not clustered, this value will be null.")
+    public List<NodeStatusSnapshotsDTO> getNodeSnapshots() {
+        return nodeSnapshots;
+    }
+
+    public void setNodeSnapshots(List<NodeStatusSnapshotsDTO> nodeSnapshots) {
+        this.nodeSnapshots = nodeSnapshots;
+    }
 }

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/StatusHistoryDetailDTO.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/StatusHistoryDetailDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java
deleted file mode 100644
index e78641e..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java
+++ /dev/null
@@ -1,59 +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;
-
-/**
- * Detail of a status history metric.
- */
-@XmlType(name = "statusHistoryDetail")
-public class StatusHistoryDetailDTO {
-
-    private String label;
-    private String value;
-
-    /**
-     * @return label for this status detail
-     */
-    @ApiModelProperty(
-            value = "The label for the status detail."
-    )
-    public String getLabel() {
-        return label;
-    }
-
-    public void setLabel(String label) {
-        this.label = label;
-    }
-
-    /**
-     * @return value for this status detail
-     */
-    @ApiModelProperty(
-            value = "The value for the status detail."
-    )
-    public String getValue() {
-        return value;
-    }
-
-    public void setValue(String value) {
-        this.value = value;
-    }
-
-}

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/entity/ClusterConnectionStatusEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java
deleted file mode 100644
index f211cc4..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java
+++ /dev/null
@@ -1,43 +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.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterConnectionStatusDTO;
-
-/**
- * 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 ClusterConnectionStatusDTO.
- */
-@XmlRootElement(name = "clusterConnectionStatusEntity")
-public class ClusterConnectionStatusEntity extends Entity {
-
-    private ClusterConnectionStatusDTO clusterConnectionStatus;
-
-    /**
-     * The ClusterConnectionStatusDTO that is being serialized.
-     *
-     * @return The ClusterConnectionStatusDTO object
-     */
-    public ClusterConnectionStatusDTO getClusterConnectionStatus() {
-        return clusterConnectionStatus;
-    }
-
-    public void setClusterConnectionStatus(ClusterConnectionStatusDTO clusterConnectionStatus) {
-        this.clusterConnectionStatus = clusterConnectionStatus;
-    }
-
-}

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/entity/ClusterProcessGroupStatusEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessGroupStatusEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessGroupStatusEntity.java
deleted file mode 100644
index f8b7e11..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessGroupStatusEntity.java
+++ /dev/null
@@ -1,43 +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.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/nifi/blob/0d3bd2c4/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java
deleted file mode 100644
index 2b8220f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java
+++ /dev/null
@@ -1,43 +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.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterProcessorStatusDTO;
-
-/**
- * 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 ClusterProcessorStatusDTO.
- */
-@XmlRootElement(name = "clusterProcessorStatusEntity")
-public class ClusterProcessorStatusEntity extends Entity {
-
-    private ClusterProcessorStatusDTO clusterProcessorStatus;
-
-    /**
-     * The ClusterProcessorStatusDTO that is being serialized.
-     *
-     * @return The ClusterProcessorStatusDTO object
-     */
-    public ClusterProcessorStatusDTO getClusterProcessorStatus() {
-        return clusterProcessorStatus;
-    }
-
-    public void setClusterProcessorStatus(ClusterProcessorStatusDTO clusterProcessorStatus) {
-        this.clusterProcessorStatus = clusterProcessorStatus;
-    }
-
-}

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/entity/ClusterRemoteProcessGroupStatusEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java
deleted file mode 100644
index 66569c5..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java
+++ /dev/null
@@ -1,44 +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.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterRemoteProcessGroupStatusDTO;
-
-/**
- * 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
- * ClusterRemoteProcessGroupStatusDTO.
- */
-@XmlRootElement(name = "clusterRemoteProcessGroupStatusEntity")
-public class ClusterRemoteProcessGroupStatusEntity extends Entity {
-
-    private ClusterRemoteProcessGroupStatusDTO clusterRemoteProcessGroupStatus;
-
-    /**
-     * The ClusterRemoteProcessGroupStatusDTO that is being serialized.
-     *
-     * @return The ClusterRemoteProcessGroupStatusDTO object
-     */
-    public ClusterRemoteProcessGroupStatusDTO getClusterRemoteProcessGroupStatus() {
-        return clusterRemoteProcessGroupStatus;
-    }
-
-    public void setClusterRemoteProcessGroupStatus(ClusterRemoteProcessGroupStatusDTO clusterRemoteProcessGroupStatus) {
-        this.clusterRemoteProcessGroupStatus = clusterRemoteProcessGroupStatus;
-    }
-
-}

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/entity/ClusterStatusHistoryEntity.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java
deleted file mode 100644
index b13984a..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java
+++ /dev/null
@@ -1,43 +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.entity;
-
-import javax.xml.bind.annotation.XmlRootElement;
-import org.apache.nifi.web.api.dto.status.ClusterStatusHistoryDTO;
-
-/**
- * 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 ClusterStatusHistoryDTO.
- */
-@XmlRootElement(name = "clusterStatusHistoryEntity")
-public class ClusterStatusHistoryEntity extends Entity {
-
-    private ClusterStatusHistoryDTO clusterStatusHistory;
-
-    /**
-     * The ClusterStatusHistoryDTO that is being serialized.
-     *
-     * @return The ClusterStatusHistoryDTO object
-     */
-    public ClusterStatusHistoryDTO getClusterStatusHistory() {
-        return clusterStatusHistory;
-    }
-
-    public void setClusterStatusHistory(ClusterStatusHistoryDTO clusterStatusHistory) {
-        this.clusterStatusHistory = clusterStatusHistory;
-    }
-
-}