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

[15/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'

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't have enough data points, even if all nodes do (which can be the case if the node performing the aggregation has a different value for the 'nifi.components.status.snapshot.frequency' property than the other nodes) - Bug fixes; code cleanup; replicate requests to bulletin board endpoint - Refactored the <Component>StatusDTO objects into <Component>StatusDTO, <Component>StatusSnapshotDTO, Node<Component>StatusSnapshotDTO objects - Introducing endpoints for accessing individual component status. - Wiring up new endpoints and updated core. - Code clean up. - Starting to handling status merging of individual components. - Nodewise breakdown has been added to Processors but the remaining components still need to be updated. - Refactor so that System Diagnostics re
 quests are replicated to nodes instead of the information being pulled from Heartbeats - Replicate request for counters instead of pulling them from heartbeats - Removed the getCounters / setCounters method from HeartbeatPayload - Implementing component specific endpoints. - Removing unused endpoints. - Supporting nodewise breakdown for system diagnostics and counters. - Updating DTOs to use more consistent naming. - Code clean up. - Addressing contrib issues. - Removed ProcessGroupStatus from HeartbeatPayload - Removing nodewise from the system diagnostics endpoint. Had included it for testing that option but did not intend for it to be committed. - Addressing comments in PR #294. - This closes #294

Signed-off-by: Matt Gilman <ma...@gmail.com>


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

Branch: refs/heads/master
Commit: 0d3bd2c401aacc5bcd602afe90883d751ff773f3
Parents: 2de7f3f
Author: Mark Payne <ma...@hotmail.com>
Authored: Thu Mar 3 13:29:34 2016 -0500
Committer: Matt Gilman <ma...@gmail.com>
Committed: Mon Apr 4 11:42:39 2016 -0400

----------------------------------------------------------------------
 .../controller/status/ProcessGroupStatus.java   |   12 -
 .../org/apache/nifi/web/api/dto/CounterDTO.java |   13 +-
 .../apache/nifi/web/api/dto/CountersDTO.java    |   50 +-
 .../nifi/web/api/dto/CountersSnapshotDTO.java   |   73 +
 .../web/api/dto/NodeCountersSnapshotDTO.java    |   78 ++
 .../web/api/dto/NodeSystemDiagnosticsDTO.java   |   59 -
 .../dto/NodeSystemDiagnosticsSnapshotDTO.java   |   77 ++
 .../nifi/web/api/dto/SystemDiagnosticsDTO.java  |  478 +------
 .../api/dto/SystemDiagnosticsSnapshotDTO.java   |  549 ++++++++
 .../dto/status/ClusterConnectionStatusDTO.java  |   94 --
 .../status/ClusterProcessGroupStatusDTO.java    |  102 --
 .../dto/status/ClusterProcessorStatusDTO.java   |  127 --
 .../ClusterRemoteProcessGroupStatusDTO.java     |   94 --
 .../api/dto/status/ClusterStatusHistoryDTO.java |   79 --
 .../web/api/dto/status/ConnectionStatusDTO.java |  180 +--
 .../dto/status/ConnectionStatusSnapshotDTO.java |  279 ++++
 .../web/api/dto/status/ControllerStatusDTO.java |  134 +-
 .../api/dto/status/NodeConnectionStatusDTO.java |   60 -
 .../status/NodeConnectionStatusSnapshotDTO.java |   78 ++
 .../dto/status/NodePortStatusSnapshotDTO.java   |   77 ++
 .../dto/status/NodeProcessGroupStatusDTO.java   |   64 -
 .../NodeProcessGroupStatusSnapshotDTO.java      |   76 +
 .../api/dto/status/NodeProcessorStatusDTO.java  |   60 -
 .../status/NodeProcessorStatusSnapshotDTO.java  |   80 ++
 .../status/NodeRemotePortStatusSnapshotDTO.java |   77 ++
 .../status/NodeRemoteProcessGroupStatusDTO.java |   60 -
 ...NodeRemoteProcessGroupStatusSnapshotDTO.java |   77 ++
 .../api/dto/status/NodeStatusHistoryDTO.java    |   60 -
 .../api/dto/status/NodeStatusSnapshotsDTO.java  |   85 ++
 .../nifi/web/api/dto/status/PortStatusDTO.java  |  115 +-
 .../api/dto/status/PortStatusSnapshotDTO.java   |  194 +++
 .../api/dto/status/ProcessGroupStatusDTO.java   |  326 +----
 .../status/ProcessGroupStatusSnapshotDTO.java   |  532 +++++++
 .../web/api/dto/status/ProcessorStatusDTO.java  |  194 +--
 .../dto/status/ProcessorStatusSnapshotDTO.java  |  301 ++++
 .../web/api/dto/status/RemotePortStatusDTO.java |   12 +-
 .../dto/status/RemoteProcessGroupStatusDTO.java |  143 +-
 .../RemoteProcessGroupStatusSnapshotDTO.java    |  215 +++
 .../nifi/web/api/dto/status/StatusDTO.java      |   45 -
 .../web/api/dto/status/StatusHistoryDTO.java    |   66 +-
 .../api/dto/status/StatusHistoryDetailDTO.java  |   59 -
 .../entity/ClusterConnectionStatusEntity.java   |   43 -
 .../entity/ClusterProcessGroupStatusEntity.java |   43 -
 .../entity/ClusterProcessorStatusEntity.java    |   43 -
 .../ClusterRemoteProcessGroupStatusEntity.java  |   44 -
 .../api/entity/ClusterStatusHistoryEntity.java  |   43 -
 .../web/api/entity/ConnectionStatusEntity.java  |   44 +
 .../api/entity/NodeSystemDiagnosticsEntity.java |   43 -
 .../nifi/web/api/entity/PortStatusEntity.java   |   44 +
 .../web/api/entity/ProcessorStatusEntity.java   |   44 +
 .../entity/RemoteProcessGroupStatusEntity.java  |   44 +
 .../cluster/protocol/NodeProtocolSender.java    |   10 -
 .../protocol/impl/NodeProtocolSenderImpl.java   |    6 -
 .../impl/NodeProtocolSenderListener.java        |    6 -
 .../protocol/jaxb/message/ObjectFactory.java    |    5 -
 .../protocol/message/NodeBulletinsMessage.java  |   42 -
 .../protocol/message/ProtocolMessage.java       |    2 -
 .../nifi/cluster/manager/ClusterManager.java    |   13 -
 .../nifi/cluster/manager/StatusMerger.java      |  646 +++++++++
 .../manager/impl/ClusteredEventAccess.java      |    2 +-
 .../cluster/manager/impl/WebClusterManager.java | 1302 ++++++++++--------
 .../org/apache/nifi/controller/Heartbeater.java |   22 -
 .../service/ControllerServiceNode.java          |   13 +-
 .../apache/nifi/cluster/HeartbeatPayload.java   |   36 +-
 .../nifi/connectable/StandardConnection.java    |    9 +-
 .../apache/nifi/controller/FlowController.java  |  206 +--
 .../nifi/controller/StandardFlowFileQueue.java  |    8 +-
 .../scheduling/StandardProcessScheduler.java    |   11 +-
 .../service/StandardControllerServiceNode.java  |   23 +-
 .../history/ConnectionStatusDescriptor.java     |  110 ++
 .../history/ProcessGroupStatusDescriptor.java   |  143 ++
 .../history/ProcessorStatusDescriptor.java      |  220 +++
 .../RemoteProcessGroupStatusDescriptor.java     |  127 ++
 .../status/history/StatusHistoryUtil.java       |    4 +-
 .../VolatileComponentStatusRepository.java      |  418 +-----
 .../nifi/events/VolatileBulletinRepository.java |   20 -
 .../nifi/cluster/HeartbeatPayloadTest.java      |   56 +-
 .../controller/TestStandardFlowFileQueue.java   |    2 +-
 .../repository/TestStandardProcessSession.java  |    2 +-
 .../TestWriteAheadFlowFileRepository.java       |    2 +-
 .../TestStandardProcessScheduler.java           |    8 +-
 .../TestStandardControllerServiceProvider.java  |    4 +-
 .../org/apache/nifi/web/NiFiServiceFacade.java  |  157 +--
 .../nifi/web/StandardNiFiServiceFacade.java     |  966 ++-----------
 .../nifi/web/api/ApplicationResource.java       |    2 +
 .../nifi/web/api/BulletinBoardResource.java     |   36 +-
 .../apache/nifi/web/api/ClusterResource.java    |  737 +---------
 .../apache/nifi/web/api/ConnectionResource.java |  112 +-
 .../apache/nifi/web/api/ControllerResource.java |  109 +-
 .../apache/nifi/web/api/InputPortResource.java  |  158 ++-
 .../org/apache/nifi/web/api/NodeResource.java   |  146 +-
 .../apache/nifi/web/api/OutputPortResource.java |  108 ++
 .../nifi/web/api/ProcessGroupResource.java      |  139 +-
 .../apache/nifi/web/api/ProcessorResource.java  |  189 ++-
 .../web/api/RemoteProcessGroupResource.java     |  170 ++-
 .../nifi/web/api/ReportingTaskResource.java     |   38 +-
 .../nifi/web/api/SystemDiagnosticsResource.java |   85 +-
 .../org/apache/nifi/web/api/dto/DtoFactory.java |  346 +++--
 .../nifi/web/controller/ControllerFacade.java   |  120 ++
 .../src/main/resources/nifi-web-api-context.xml |    4 +
 .../cluster-connection-summary-dialog.jsp       |    1 +
 .../cluster-input-port-summary-dialog.jsp       |    1 +
 .../cluster-output-port-summary-dialog.jsp      |    1 +
 .../cluster-processor-summary-dialog.jsp        |    1 +
 ...ster-remote-process-group-summary-dialog.jsp |    1 +
 .../src/main/webapp/js/nf/canvas/nf-actions.js  |   28 +-
 .../src/main/webapp/js/nf/canvas/nf-canvas.js   |    3 +-
 .../src/main/webapp/js/nf/canvas/nf-graph.js    |   26 +-
 .../webapp/js/nf/counters/nf-counters-table.js  |    9 +-
 .../src/main/webapp/js/nf/nf-status-history.js  |  412 ++----
 .../webapp/js/nf/summary/nf-cluster-search.js   |   10 +-
 .../webapp/js/nf/summary/nf-summary-table.js    |  434 +++---
 .../src/main/webapp/js/nf/summary/nf-summary.js |    4 +-
 113 files changed, 7297 insertions(+), 6703 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/0d3bd2c4/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
----------------------------------------------------------------------
diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
index eb0339f..db16954 100644
--- a/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
+++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/ProcessGroupStatus.java
@@ -32,7 +32,6 @@ public class ProcessGroupStatus implements Cloneable {
     private Long inputContentSize;
     private Integer outputCount;
     private Long outputContentSize;
-    private long creationTimestamp;
     private Integer activeThreadCount;
     private Integer queuedCount;
     private Long queuedContentSize;
@@ -132,14 +131,6 @@ public class ProcessGroupStatus implements Cloneable {
         this.queuedContentSize = queuedContentSize;
     }
 
-    public long getCreationTimestamp() {
-        return creationTimestamp;
-    }
-
-    public void setCreationTimestamp(final long creationTimestamp) {
-        this.creationTimestamp = creationTimestamp;
-    }
-
     public Integer getActiveThreadCount() {
         return activeThreadCount;
     }
@@ -249,7 +240,6 @@ public class ProcessGroupStatus implements Cloneable {
 
         final ProcessGroupStatus clonedObj = new ProcessGroupStatus();
 
-        clonedObj.creationTimestamp = creationTimestamp;
         clonedObj.id = id;
         clonedObj.name = name;
         clonedObj.outputContentSize = outputContentSize;
@@ -332,8 +322,6 @@ public class ProcessGroupStatus implements Cloneable {
         builder.append(outputCount);
         builder.append(", outputBytes=");
         builder.append(outputContentSize);
-        builder.append(", creationTimestamp=");
-        builder.append(creationTimestamp);
         builder.append(", activeThreadCount=");
         builder.append(activeThreadCount);
         builder.append(", flowFilesTransferred=");

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/CounterDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java
index 615ad93..b3315b9 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java
@@ -17,13 +17,14 @@
 package org.apache.nifi.web.api.dto;
 
 import com.wordnik.swagger.annotations.ApiModelProperty;
+
 import javax.xml.bind.annotation.XmlType;
 
 /**
  * Counter value for a specific component in a specific context. A counter is a value that a component can adjust during processing.
  */
 @XmlType(name = "counter")
-public class CounterDTO {
+public class CounterDTO implements Cloneable {
 
     private String id;
     private String context;
@@ -98,4 +99,14 @@ public class CounterDTO {
         this.valueCount = valueCount;
     }
 
+    @Override
+    public CounterDTO clone() {
+        final CounterDTO other = new CounterDTO();
+        other.setId(getId());
+        other.setName(getName());
+        other.setContext(getContext());
+        other.setValue(getValue());
+        other.setValueCount(getValueCount());
+        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/CountersDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java
index 0f162c9..e21d331 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java
@@ -14,51 +14,37 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.nifi.web.api.dto;
 
-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 org.apache.nifi.web.api.dto.util.TimeAdapter;
 
-/**
- * All the counters in this NiFi instance at a given time.
- */
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
 @XmlType(name = "counters")
 public class CountersDTO {
+    private CountersSnapshotDTO aggregateSnapshot;
+    private List<NodeCountersSnapshotDTO> nodeSnapshots;
 
-    private Date generated;
-    private Collection<CounterDTO> counters;
-
-    /**
-     * @return the collection of counters
-     */
-    @ApiModelProperty(
-            value = "All counters in the NiFi."
-    )
-    public Collection<CounterDTO> getCounters() {
-        return counters;
+    @ApiModelProperty("A Counters snapshot that represents the aggregate values 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 CountersSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
     }
 
-    public void setCounters(Collection<CounterDTO> counters) {
-        this.counters = counters;
+    public void setAggregateSnapshot(CountersSnapshotDTO aggregateSnapshot) {
+        this.aggregateSnapshot = aggregateSnapshot;
     }
 
-    /**
-     * @return the date/time that this report was generated
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    @ApiModelProperty(
-            value = "The timestamp when the report was generated."
-    )
-    public Date getGenerated() {
-        return generated;
+    @ApiModelProperty("A Counters 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<NodeCountersSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
     }
 
-    public void setGenerated(Date generated) {
-        this.generated = generated;
+    public void setNodeSnapshots(List<NodeCountersSnapshotDTO> 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/CountersSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersSnapshotDTO.java
new file mode 100644
index 0000000..1e1b389
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersSnapshotDTO.java
@@ -0,0 +1,73 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+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 org.apache.nifi.web.api.dto.util.TimeAdapter;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+
+/**
+ * All the counters in this NiFi instance at a given time.
+ */
+@XmlType(name = "countersSnapshot")
+public class CountersSnapshotDTO implements Cloneable {
+
+    private Date generated;
+    private Collection<CounterDTO> counters;
+
+    @ApiModelProperty("All counters in the NiFi.")
+    public Collection<CounterDTO> getCounters() {
+        return counters;
+    }
+
+    public void setCounters(Collection<CounterDTO> counters) {
+        this.counters = counters;
+    }
+
+    @XmlJavaTypeAdapter(TimeAdapter.class)
+    @ApiModelProperty("The timestamp when the report was generated.")
+    public Date getGenerated() {
+        return generated;
+    }
+
+    public void setGenerated(Date generated) {
+        this.generated = generated;
+    }
+
+    @Override
+    public CountersSnapshotDTO clone() {
+        final CountersSnapshotDTO other = new CountersSnapshotDTO();
+        other.setGenerated(getGenerated());
+
+        final List<CounterDTO> clonedCounters = new ArrayList<>(getCounters().size());
+        for (final CounterDTO counterDto : getCounters()) {
+            clonedCounters.add(counterDto.clone());
+        }
+
+        other.setCounters(clonedCounters);
+        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/NodeCountersSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeCountersSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeCountersSnapshotDTO.java
new file mode 100644
index 0000000..b7f19c7
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeCountersSnapshotDTO.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;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "nodeProcessorStatusSnapshot")
+public class NodeCountersSnapshotDTO implements Cloneable {
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private CountersSnapshotDTO snapshot;
+
+    @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 counters from the node.")
+    public CountersSnapshotDTO getSnapshot() {
+        return snapshot;
+    }
+
+    public void setSnapshot(CountersSnapshotDTO snapshot) {
+        this.snapshot = snapshot;
+    }
+
+    @Override
+    public NodeCountersSnapshotDTO clone() {
+        final NodeCountersSnapshotDTO other = new NodeCountersSnapshotDTO();
+        other.setNodeId(getNodeId());
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setSnapshot(getSnapshot().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/NodeSystemDiagnosticsDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java
deleted file mode 100644
index 8f925aa..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.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;
-
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import javax.xml.bind.annotation.XmlType;
-
-/**
- * The system diagnostics for a node with this NiFi cluster.
- */
-@XmlType(name = "nodeSystemDiagnostics")
-public class NodeSystemDiagnosticsDTO {
-
-    private NodeDTO node;
-    private SystemDiagnosticsDTO systemDiagnostics;
-
-    /**
-     * @return the node
-     */
-    @ApiModelProperty(
-            value = "The node."
-    )
-    public NodeDTO getNode() {
-        return node;
-    }
-
-    public void setNode(NodeDTO node) {
-        this.node = node;
-    }
-
-    /**
-     * @return the system diagnostics
-     */
-    @ApiModelProperty(
-            value = "The diagnostics for the system the node is on."
-    )
-    public SystemDiagnosticsDTO getSystemDiagnostics() {
-        return systemDiagnostics;
-    }
-
-    public void setControllerStatus(SystemDiagnosticsDTO systemDiagnostics) {
-        this.systemDiagnostics = systemDiagnostics;
-    }
-
-}

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/NodeSystemDiagnosticsSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsSnapshotDTO.java
new file mode 100644
index 0000000..f27b05d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsSnapshotDTO.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;
+
+import javax.xml.bind.annotation.XmlType;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+@XmlType(name = "nodeSystemDiagnosticsSnapshot")
+public class NodeSystemDiagnosticsSnapshotDTO implements Cloneable {
+    private String nodeId;
+    private String address;
+    private Integer apiPort;
+
+    private SystemDiagnosticsSnapshotDTO snapshot;
+
+    @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 System Diagnostics snapshot from the node.")
+    public SystemDiagnosticsSnapshotDTO getSnapshot() {
+        return snapshot;
+    }
+
+    public void setSnapshot(SystemDiagnosticsSnapshotDTO snapshot) {
+        this.snapshot = snapshot;
+    }
+
+    @Override
+    public NodeSystemDiagnosticsSnapshotDTO clone() {
+        final NodeSystemDiagnosticsSnapshotDTO other = new NodeSystemDiagnosticsSnapshotDTO();
+        other.setAddress(getAddress());
+        other.setApiPort(getApiPort());
+        other.setNodeId(getNodeId());
+        other.setSnapshot(getSnapshot().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/SystemDiagnosticsDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java
index d2a9d1a..40a04d4 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java
@@ -14,478 +14,38 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.nifi.web.api.dto;
 
-import com.wordnik.swagger.annotations.ApiModelProperty;
-import java.util.Date;
-import java.util.Set;
+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;
 
-/**
- * The diagnostics of the system this NiFi is running on.
- */
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
 @XmlType(name = "systemDiagnostics")
 public class SystemDiagnosticsDTO {
+    private SystemDiagnosticsSnapshotDTO aggregateSnapshot;
+    private List<NodeSystemDiagnosticsSnapshotDTO> nodeSnapshots;
 
-    private String totalNonHeap;
-    private String usedNonHeap;
-    private String freeNonHeap;
-    private String maxNonHeap;
-    private String nonHeapUtilization;
-
-    private String totalHeap;
-    private String usedHeap;
-    private String freeHeap;
-    private String maxHeap;
-    private String heapUtilization;
-
-    private Integer availableProcessors;
-    private Double processorLoadAverage;
-
-    private Integer totalThreads;
-    private Integer daemonThreads;
-
-    private StorageUsageDTO flowFileRepositoryStorageUsage;
-    private Set<StorageUsageDTO> contentRepositoryStorageUsage;
-    private Set<GarbageCollectionDTO> garbageCollection;
-
-    private Date statsLastRefreshed;
-
-    /* getters / setters */
-    /**
-     * @return number of available processors, if supported
-     */
-    @ApiModelProperty(
-            value = "Number of available processors if supported by the underlying system."
-    )
-    public Integer getAvailableProcessors() {
-        return availableProcessors;
-    }
-
-    public void setAvailableProcessors(Integer availableProcessors) {
-        this.availableProcessors = availableProcessors;
-    }
-
-    /**
-     * @return number of daemon threads
-     */
-    @ApiModelProperty(
-            value = "Number of daemon threads."
-    )
-    public Integer getDaemonThreads() {
-        return daemonThreads;
-    }
-
-    public void setDaemonThreads(Integer daemonThreads) {
-        this.daemonThreads = daemonThreads;
-    }
-
-    /**
-     * @return amount of free heap
-     */
-    @ApiModelProperty(
-            value = "Amount of free heap."
-    )
-    public String getFreeHeap() {
-        return freeHeap;
-    }
-
-    public void setFreeHeap(String freeHeap) {
-        this.freeHeap = freeHeap;
-    }
-
-    /**
-     * @return amount of free non-heap
-     */
-    @ApiModelProperty(
-            value = "Amount of free non heap."
-    )
-    public String getFreeNonHeap() {
-        return freeNonHeap;
-    }
-
-    public void setFreeNonHeap(String freeNonHeap) {
-        this.freeNonHeap = freeNonHeap;
-    }
-
-    /**
-     * @return max size of the heap
-     */
-    @ApiModelProperty(
-            value = "Maximum size of heap."
-    )
-    public String getMaxHeap() {
-        return maxHeap;
-    }
-
-    public void setMaxHeap(String maxHeap) {
-        this.maxHeap = maxHeap;
-    }
-
-    /**
-     * @return max size of the non-heap
-     */
-    @ApiModelProperty(
-            value = "Maximum size of non heap."
-    )
-    public String getMaxNonHeap() {
-        return maxNonHeap;
-    }
-
-    public void setMaxNonHeap(String maxNonHeap) {
-        this.maxNonHeap = maxNonHeap;
-    }
-
-    /**
-     * @return processor load average, if supported
-     */
-    @ApiModelProperty(
-            value = "The processor load average if supported by the underlying system."
-    )
-    public Double getProcessorLoadAverage() {
-        return processorLoadAverage;
-    }
-
-    public void setProcessorLoadAverage(Double processorLoadAverage) {
-        this.processorLoadAverage = processorLoadAverage;
-    }
-
-    /**
-     * @return total size of the heap
-     */
-    @ApiModelProperty(
-            value = "Total size of heap."
-    )
-    public String getTotalHeap() {
-        return totalHeap;
-    }
-
-    public void setTotalHeap(String totalHeap) {
-        this.totalHeap = totalHeap;
-    }
-
-    /**
-     * @return total size of non-heap
-     */
-    @ApiModelProperty(
-            value = "Total size of non heap."
-    )
-    public String getTotalNonHeap() {
-        return totalNonHeap;
-    }
-
-    public void setTotalNonHeap(String totalNonHeap) {
-        this.totalNonHeap = totalNonHeap;
-    }
-
-    /**
-     * @return total number of threads
-     */
-    @ApiModelProperty(
-            value = "Total number of threads."
-    )
-    public Integer getTotalThreads() {
-        return totalThreads;
-    }
-
-    public void setTotalThreads(Integer totalThreads) {
-        this.totalThreads = totalThreads;
-    }
-
-    /**
-     * @return amount of used heap
-     */
-    @ApiModelProperty(
-            value = "Amount of used heap."
-    )
-    public String getUsedHeap() {
-        return usedHeap;
-    }
-
-    public void setUsedHeap(String usedHeap) {
-        this.usedHeap = usedHeap;
-    }
-
-    /**
-     * @return amount of used non-heap
-     */
-    @ApiModelProperty(
-            value = "Amount of use non heap."
-    )
-    public String getUsedNonHeap() {
-        return usedNonHeap;
-    }
-
-    public void setUsedNonHeap(String usedNonHeap) {
-        this.usedNonHeap = usedNonHeap;
-    }
-
-    /**
-     * @return heap utilization
-     */
-    @ApiModelProperty(
-            value = "Utilization of heap."
-    )
-    public String getHeapUtilization() {
-        return heapUtilization;
-    }
-
-    public void setHeapUtilization(String heapUtilization) {
-        this.heapUtilization = heapUtilization;
-    }
-
-    /**
-     * @return non-heap utilization
-     */
-    @ApiModelProperty(
-            value = "Utilization of non heap."
-    )
-    public String getNonHeapUtilization() {
-        return nonHeapUtilization;
-    }
-
-    public void setNonHeapUtilization(String nonHeapUsage) {
-        this.nonHeapUtilization = nonHeapUsage;
-    }
-
-    /**
-     * @return content repository storage usage
-     */
-    @ApiModelProperty(
-            value = "The content repository storage usage."
-    )
-    public Set<StorageUsageDTO> getContentRepositoryStorageUsage() {
-        return contentRepositoryStorageUsage;
-    }
-
-    public void setContentRepositoryStorageUsage(Set<StorageUsageDTO> contentRepositoryStorageUsage) {
-        this.contentRepositoryStorageUsage = contentRepositoryStorageUsage;
-    }
-
-    /**
-     * @return flowfile repository storage usage
-     */
-    @ApiModelProperty(
-            value = "The flowfile repository storage usage."
-    )
-    public StorageUsageDTO getFlowFileRepositoryStorageUsage() {
-        return flowFileRepositoryStorageUsage;
-    }
-
-    public void setFlowFileRepositoryStorageUsage(StorageUsageDTO flowFileRepositoryStorageUsage) {
-        this.flowFileRepositoryStorageUsage = flowFileRepositoryStorageUsage;
-    }
-
-    /**
-     * @return Garbage collection details
-     */
-    @ApiModelProperty(
-            value = "The garbage collection details."
-    )
-    public Set<GarbageCollectionDTO> getGarbageCollection() {
-        return garbageCollection;
-    }
-
-    public void setGarbageCollection(Set<GarbageCollectionDTO> garbageCollection) {
-        this.garbageCollection = garbageCollection;
-    }
 
-    /**
-     * @return When these diagnostics were generated
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    @ApiModelProperty(
-            value = "When the diagnostics were generated."
-    )
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
+    @ApiModelProperty("A systems diagnostic snapshot that represents the aggregate values 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 SystemDiagnosticsSnapshotDTO getAggregateSnapshot() {
+        return aggregateSnapshot;
     }
 
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
+    public void setAggregateSnapshot(SystemDiagnosticsSnapshotDTO aggregateSnapshot) {
+        this.aggregateSnapshot = aggregateSnapshot;
     }
 
-    /**
-     * Details of storage usage.
-     */
-    @XmlType(name = "storageUsage")
-    public static class StorageUsageDTO {
-
-        private String identifier;
-        private String freeSpace;
-        private String totalSpace;
-        private String usedSpace;
-        private Long freeSpaceBytes;
-        private Long totalSpaceBytes;
-        private Long usedSpaceBytes;
-        private String utilization;
-
-        /**
-         * @return identifier for this storage location
-         */
-        @ApiModelProperty(
-                value = "The identifier of this storage location. The identifier will correspond to the identifier keyed in the storage configuration."
-        )
-        public String getIdentifier() {
-            return identifier;
-        }
-
-        public void setIdentifier(String identifier) {
-            this.identifier = identifier;
-        }
-
-        /**
-         * @return amount of free space
-         */
-        @ApiModelProperty(
-                value = "Amount of free space."
-        )
-        public String getFreeSpace() {
-            return freeSpace;
-        }
-
-        public void setFreeSpace(String freeSpace) {
-            this.freeSpace = freeSpace;
-        }
-
-        /**
-         * @return freeSpace amount of total space
-         */
-        @ApiModelProperty(
-                value = "Amount of total space."
-        )
-        public String getTotalSpace() {
-            return totalSpace;
-        }
-
-        public void setTotalSpace(String totalSpace) {
-            this.totalSpace = totalSpace;
-        }
-
-        /**
-         * @return amount of used space
-         */
-        @ApiModelProperty(
-                value = "Amount of used space."
-        )
-        public String getUsedSpace() {
-            return usedSpace;
-        }
-
-        public void setUsedSpace(String usedSpace) {
-            this.usedSpace = usedSpace;
-        }
-
-        /**
-         * @return utilization of this storage location
-         */
-        @ApiModelProperty(
-                value = "Utilization of this storage location."
-        )
-        public String getUtilization() {
-            return utilization;
-        }
-
-        public void setUtilization(String utilization) {
-            this.utilization = utilization;
-        }
-
-        /**
-         * @return number of bytes of free space
-         */
-        @ApiModelProperty(
-                value = "The number of bytes of free space."
-        )
-        public Long getFreeSpaceBytes() {
-            return freeSpaceBytes;
-        }
-
-        public void setFreeSpaceBytes(Long freeSpaceBytes) {
-            this.freeSpaceBytes = freeSpaceBytes;
-        }
-
-        /**
-         * @return number of bytes of total space
-         */
-        @ApiModelProperty(
-                value = "The number of bytes of total space."
-        )
-        public Long getTotalSpaceBytes() {
-            return totalSpaceBytes;
-        }
-
-        public void setTotalSpaceBytes(Long totalSpaceBytes) {
-            this.totalSpaceBytes = totalSpaceBytes;
-        }
-
-        /**
-         * @return number of bytes of used space
-         */
-        @ApiModelProperty(
-                value = "The number of bytes of used space."
-        )
-        public Long getUsedSpaceBytes() {
-            return usedSpaceBytes;
-        }
-
-        public void setUsedSpaceBytes(Long usedSpaceBytes) {
-            this.usedSpaceBytes = usedSpaceBytes;
-        }
+    @ApiModelProperty("A systems diagnostics 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<NodeSystemDiagnosticsSnapshotDTO> getNodeSnapshots() {
+        return nodeSnapshots;
     }
 
-    /**
-     * Details for garbage collection.
-     */
-    @XmlType(name = "garbageCollection")
-    public static class GarbageCollectionDTO {
-
-        private String name;
-        private long collectionCount;
-        private String collectionTime;
-
-        /**
-         * @return name of the garbage collector
-         */
-        @ApiModelProperty(
-                value = "The name of the garbage collector."
-        )
-        public String getName() {
-            return name;
-        }
-
-        public void setName(String name) {
-            this.name = name;
-        }
-
-        @ApiModelProperty(
-                value = "The number of times garbage collection has run."
-        )
-        public long getCollectionCount() {
-            return collectionCount;
-        }
-
-        /**
-         * @param collectionCount number of times garbage collection has run
-         */
-        public void setCollectionCount(long collectionCount) {
-            this.collectionCount = collectionCount;
-        }
-
-        /**
-         * @return total amount of time spent garbage collecting
-         */
-        @ApiModelProperty(
-                value = "The total amount of time spent garbage collecting."
-        )
-        public String getCollectionTime() {
-            return collectionTime;
-        }
-
-        public void setCollectionTime(String collectionTime) {
-            this.collectionTime = collectionTime;
-        }
-
+    public void setNodeSnapshots(List<NodeSystemDiagnosticsSnapshotDTO> 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/SystemDiagnosticsSnapshotDTO.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsSnapshotDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsSnapshotDTO.java
new file mode 100644
index 0000000..7305837
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsSnapshotDTO.java
@@ -0,0 +1,549 @@
+/*
+ * 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;
+
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.xml.bind.annotation.XmlType;
+import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
+
+import org.apache.nifi.web.api.dto.util.TimeAdapter;
+
+import com.wordnik.swagger.annotations.ApiModelProperty;
+
+/**
+ * The diagnostics of the system this NiFi is running on.
+ */
+@XmlType(name = "systemDiagnosticsSnapshot")
+public class SystemDiagnosticsSnapshotDTO implements Cloneable {
+
+    private String totalNonHeap;
+    private Long totalNonHeapBytes;
+    private String usedNonHeap;
+    private Long usedNonHeapBytes;
+    private String freeNonHeap;
+    private Long freeNonHeapBytes;
+    private String maxNonHeap;
+    private Long maxNonHeapBytes;
+    private String nonHeapUtilization;
+
+    private String totalHeap;
+    private Long totalHeapBytes;
+    private String usedHeap;
+    private Long usedHeapBytes;
+    private String freeHeap;
+    private Long freeHeapBytes;
+    private String maxHeap;
+    private Long maxHeapBytes;
+    private String heapUtilization;
+
+    private Integer availableProcessors;
+    private Double processorLoadAverage;
+
+    private Integer totalThreads;
+    private Integer daemonThreads;
+
+    private StorageUsageDTO flowFileRepositoryStorageUsage;
+    private Set<StorageUsageDTO> contentRepositoryStorageUsage;
+    private Set<GarbageCollectionDTO> garbageCollection;
+
+    private Date statsLastRefreshed;
+
+
+    @ApiModelProperty("Number of available processors if supported by the underlying system.")
+    public Integer getAvailableProcessors() {
+        return availableProcessors;
+    }
+
+    public void setAvailableProcessors(Integer availableProcessors) {
+        this.availableProcessors = availableProcessors;
+    }
+
+    @ApiModelProperty("Number of daemon threads.")
+    public Integer getDaemonThreads() {
+        return daemonThreads;
+    }
+
+    public void setDaemonThreads(Integer daemonThreads) {
+        this.daemonThreads = daemonThreads;
+    }
+
+    @ApiModelProperty("Amount of free heap.")
+    public String getFreeHeap() {
+        return freeHeap;
+    }
+
+    public void setFreeHeap(String freeHeap) {
+        this.freeHeap = freeHeap;
+    }
+
+    @ApiModelProperty("Amount of free non heap.")
+    public String getFreeNonHeap() {
+        return freeNonHeap;
+    }
+
+    public void setFreeNonHeap(String freeNonHeap) {
+        this.freeNonHeap = freeNonHeap;
+    }
+
+    @ApiModelProperty("Maximum size of heap.")
+    public String getMaxHeap() {
+        return maxHeap;
+    }
+
+    public void setMaxHeap(String maxHeap) {
+        this.maxHeap = maxHeap;
+    }
+
+    @ApiModelProperty("Maximum size of non heap.")
+    public String getMaxNonHeap() {
+        return maxNonHeap;
+    }
+
+    public void setMaxNonHeap(String maxNonHeap) {
+        this.maxNonHeap = maxNonHeap;
+    }
+
+    @ApiModelProperty("The processor load average if supported by the underlying system.")
+    public Double getProcessorLoadAverage() {
+        return processorLoadAverage;
+    }
+
+    public void setProcessorLoadAverage(Double processorLoadAverage) {
+        this.processorLoadAverage = processorLoadAverage;
+    }
+
+    @ApiModelProperty("Total size of heap.")
+    public String getTotalHeap() {
+        return totalHeap;
+    }
+
+    public void setTotalHeap(String totalHeap) {
+        this.totalHeap = totalHeap;
+    }
+
+    @ApiModelProperty("Total size of non heap.")
+    public String getTotalNonHeap() {
+        return totalNonHeap;
+    }
+
+    public void setTotalNonHeap(String totalNonHeap) {
+        this.totalNonHeap = totalNonHeap;
+    }
+
+    @ApiModelProperty("Total number of threads.")
+    public Integer getTotalThreads() {
+        return totalThreads;
+    }
+
+    public void setTotalThreads(Integer totalThreads) {
+        this.totalThreads = totalThreads;
+    }
+
+    @ApiModelProperty("Amount of used heap.")
+    public String getUsedHeap() {
+        return usedHeap;
+    }
+
+    public void setUsedHeap(String usedHeap) {
+        this.usedHeap = usedHeap;
+    }
+
+    @ApiModelProperty("Amount of use non heap.")
+    public String getUsedNonHeap() {
+        return usedNonHeap;
+    }
+
+    public void setUsedNonHeap(String usedNonHeap) {
+        this.usedNonHeap = usedNonHeap;
+    }
+
+    @ApiModelProperty("Utilization of heap.")
+    public String getHeapUtilization() {
+        return heapUtilization;
+    }
+
+    public void setHeapUtilization(String heapUtilization) {
+        this.heapUtilization = heapUtilization;
+    }
+
+    @ApiModelProperty("Utilization of non heap.")
+    public String getNonHeapUtilization() {
+        return nonHeapUtilization;
+    }
+
+    public void setNonHeapUtilization(String nonHeapUsage) {
+        this.nonHeapUtilization = nonHeapUsage;
+    }
+
+    @ApiModelProperty("The content repository storage usage.")
+    public Set<StorageUsageDTO> getContentRepositoryStorageUsage() {
+        return contentRepositoryStorageUsage;
+    }
+
+    public void setContentRepositoryStorageUsage(Set<StorageUsageDTO> contentRepositoryStorageUsage) {
+        this.contentRepositoryStorageUsage = contentRepositoryStorageUsage;
+    }
+
+    @ApiModelProperty("The flowfile repository storage usage.")
+    public StorageUsageDTO getFlowFileRepositoryStorageUsage() {
+        return flowFileRepositoryStorageUsage;
+    }
+
+    public void setFlowFileRepositoryStorageUsage(StorageUsageDTO flowFileRepositoryStorageUsage) {
+        this.flowFileRepositoryStorageUsage = flowFileRepositoryStorageUsage;
+    }
+
+    @ApiModelProperty("The garbage collection details.")
+    public Set<GarbageCollectionDTO> getGarbageCollection() {
+        return garbageCollection;
+    }
+
+    public void setGarbageCollection(Set<GarbageCollectionDTO> garbageCollection) {
+        this.garbageCollection = garbageCollection;
+    }
+
+    @XmlJavaTypeAdapter(TimeAdapter.class)
+    @ApiModelProperty("When the diagnostics were generated.")
+    public Date getStatsLastRefreshed() {
+        return statsLastRefreshed;
+    }
+
+    public void setStatsLastRefreshed(Date statsLastRefreshed) {
+        this.statsLastRefreshed = statsLastRefreshed;
+    }
+
+
+    @ApiModelProperty("Total number of bytes allocated to the JVM not used for heap")
+    public Long getTotalNonHeapBytes() {
+        return totalNonHeapBytes;
+    }
+
+    public void setTotalNonHeapBytes(Long totalNonHeapBytes) {
+        this.totalNonHeapBytes = totalNonHeapBytes;
+    }
+
+    @ApiModelProperty("Total number of bytes used by the JVM not in the heap space")
+    public Long getUsedNonHeapBytes() {
+        return usedNonHeapBytes;
+    }
+
+    public void setUsedNonHeapBytes(Long usedNonHeapBytes) {
+        this.usedNonHeapBytes = usedNonHeapBytes;
+    }
+
+    @ApiModelProperty("Total number of free non-heap bytes available to the JVM")
+    public Long getFreeNonHeapBytes() {
+        return freeNonHeapBytes;
+    }
+
+    public void setFreeNonHeapBytes(Long freeNonHeapBytes) {
+        this.freeNonHeapBytes = freeNonHeapBytes;
+    }
+
+    @ApiModelProperty("The maximum number of bytes that the JVM can use for non-heap purposes")
+    public Long getMaxNonHeapBytes() {
+        return maxNonHeapBytes;
+    }
+
+    public void setMaxNonHeapBytes(Long maxNonHeapBytes) {
+        this.maxNonHeapBytes = maxNonHeapBytes;
+    }
+
+    @ApiModelProperty("The total number of bytes that are available for the JVM heap to use")
+    public Long getTotalHeapBytes() {
+        return totalHeapBytes;
+    }
+
+    public void setTotalHeapBytes(Long totalHeapBytes) {
+        this.totalHeapBytes = totalHeapBytes;
+    }
+
+    @ApiModelProperty("The number of bytes of JVM heap that are currently being used")
+    public Long getUsedHeapBytes() {
+        return usedHeapBytes;
+    }
+
+    public void setUsedHeapBytes(Long usedHeapBytes) {
+        this.usedHeapBytes = usedHeapBytes;
+    }
+
+    @ApiModelProperty("The number of bytes that are allocated to the JVM heap but not currently being used")
+    public Long getFreeHeapBytes() {
+        return freeHeapBytes;
+    }
+
+    public void setFreeHeapBytes(Long freeHeapBytes) {
+        this.freeHeapBytes = freeHeapBytes;
+    }
+
+    @ApiModelProperty("The maximum number of bytes that can be used by the JVM")
+    public Long getMaxHeapBytes() {
+        return maxHeapBytes;
+    }
+
+    public void setMaxHeapBytes(Long maxHeapBytes) {
+        this.maxHeapBytes = maxHeapBytes;
+    }
+
+
+    @Override
+    public SystemDiagnosticsSnapshotDTO clone() {
+        final SystemDiagnosticsSnapshotDTO other = new SystemDiagnosticsSnapshotDTO();
+        other.setAvailableProcessors(getAvailableProcessors());
+        other.setDaemonThreads(getDaemonThreads());
+        other.setFreeHeap(getFreeHeap());
+        other.setFreeHeapBytes(getFreeHeapBytes());
+        other.setFreeNonHeap(getFreeNonHeap());
+        other.setFreeNonHeapBytes(getFreeNonHeapBytes());
+        other.setHeapUtilization(getHeapUtilization());
+        other.setMaxHeap(getMaxHeap());
+        other.setMaxHeapBytes(getMaxHeapBytes());
+        other.setMaxNonHeap(getMaxNonHeap());
+        other.setMaxNonHeapBytes(getMaxNonHeapBytes());
+        other.setNonHeapUtilization(getNonHeapUtilization());
+        other.setProcessorLoadAverage(getProcessorLoadAverage());
+        other.setStatsLastRefreshed(getStatsLastRefreshed());
+        other.setTotalHeap(getTotalHeap());
+        other.setTotalHeapBytes(getTotalHeapBytes());
+        other.setTotalNonHeap(getTotalNonHeap());
+        other.setTotalNonHeapBytes(getTotalNonHeapBytes());
+        other.setTotalThreads(getTotalThreads());
+        other.setUsedHeap(getUsedHeap());
+        other.setUsedHeapBytes(getUsedHeapBytes());
+        other.setUsedNonHeap(getUsedNonHeap());
+        other.setUsedNonHeapBytes(getUsedNonHeapBytes());
+
+        other.setFlowFileRepositoryStorageUsage(getFlowFileRepositoryStorageUsage().clone());
+
+        final Set<StorageUsageDTO> contentRepoStorageUsage = new HashSet<>();
+        other.setContentRepositoryStorageUsage(contentRepoStorageUsage);
+        for (final StorageUsageDTO usage : getContentRepositoryStorageUsage()) {
+            contentRepoStorageUsage.add(usage.clone());
+        }
+
+        final Set<GarbageCollectionDTO> gcUsage = new HashSet<>();
+        other.setGarbageCollection(gcUsage);
+        for (final GarbageCollectionDTO gcDto : getGarbageCollection()) {
+            gcUsage.add(gcDto.clone());
+        }
+
+        return other;
+    }
+
+    /**
+     * Details of storage usage.
+     */
+    @XmlType(name = "storageUsage")
+    public static class StorageUsageDTO implements Cloneable {
+
+        private String identifier;
+        private String freeSpace;
+        private String totalSpace;
+        private String usedSpace;
+        private Long freeSpaceBytes;
+        private Long totalSpaceBytes;
+        private Long usedSpaceBytes;
+        private String utilization;
+
+        /**
+         * @return identifier for this storage location
+         */
+        @ApiModelProperty(
+                value = "The identifier of this storage location. The identifier will correspond to the identifier keyed in the storage configuration."
+        )
+        public String getIdentifier() {
+            return identifier;
+        }
+
+        public void setIdentifier(String identifier) {
+            this.identifier = identifier;
+        }
+
+        /**
+         * @return amount of free space
+         */
+        @ApiModelProperty(
+                value = "Amount of free space."
+        )
+        public String getFreeSpace() {
+            return freeSpace;
+        }
+
+        public void setFreeSpace(String freeSpace) {
+            this.freeSpace = freeSpace;
+        }
+
+        /**
+         * @return freeSpace amount of total space
+         */
+        @ApiModelProperty(
+                value = "Amount of total space."
+        )
+        public String getTotalSpace() {
+            return totalSpace;
+        }
+
+        public void setTotalSpace(String totalSpace) {
+            this.totalSpace = totalSpace;
+        }
+
+        /**
+         * @return amount of used space
+         */
+        @ApiModelProperty(
+                value = "Amount of used space."
+        )
+        public String getUsedSpace() {
+            return usedSpace;
+        }
+
+        public void setUsedSpace(String usedSpace) {
+            this.usedSpace = usedSpace;
+        }
+
+        /**
+         * @return utilization of this storage location
+         */
+        @ApiModelProperty(
+                value = "Utilization of this storage location."
+        )
+        public String getUtilization() {
+            return utilization;
+        }
+
+        public void setUtilization(String utilization) {
+            this.utilization = utilization;
+        }
+
+        /**
+         * @return number of bytes of free space
+         */
+        @ApiModelProperty(
+                value = "The number of bytes of free space."
+        )
+        public Long getFreeSpaceBytes() {
+            return freeSpaceBytes;
+        }
+
+        public void setFreeSpaceBytes(Long freeSpaceBytes) {
+            this.freeSpaceBytes = freeSpaceBytes;
+        }
+
+        /**
+         * @return number of bytes of total space
+         */
+        @ApiModelProperty(
+                value = "The number of bytes of total space."
+        )
+        public Long getTotalSpaceBytes() {
+            return totalSpaceBytes;
+        }
+
+        public void setTotalSpaceBytes(Long totalSpaceBytes) {
+            this.totalSpaceBytes = totalSpaceBytes;
+        }
+
+        /**
+         * @return number of bytes of used space
+         */
+        @ApiModelProperty(
+                value = "The number of bytes of used space."
+        )
+        public Long getUsedSpaceBytes() {
+            return usedSpaceBytes;
+        }
+
+        public void setUsedSpaceBytes(Long usedSpaceBytes) {
+            this.usedSpaceBytes = usedSpaceBytes;
+        }
+
+        @Override
+        public StorageUsageDTO clone() {
+            final StorageUsageDTO other = new StorageUsageDTO();
+            other.setIdentifier(getIdentifier());
+            other.setFreeSpace(getFreeSpace());
+            other.setTotalSpace(getTotalSpace());
+            other.setUsedSpace(getUsedSpace());
+            other.setFreeSpaceBytes(getFreeSpaceBytes());
+            other.setTotalSpaceBytes(getTotalSpaceBytes());
+            other.setUsedSpaceBytes(getUsedSpaceBytes());
+            other.setUtilization(getUtilization());
+            return other;
+        }
+    }
+
+    /**
+     * Details for garbage collection.
+     */
+    @XmlType(name = "garbageCollection")
+    public static class GarbageCollectionDTO implements Cloneable {
+
+        private String name;
+        private long collectionCount;
+        private String collectionTime;
+        private Long collectionMillis;
+
+        @ApiModelProperty("The name of the garbage collector.")
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        @ApiModelProperty("The number of times garbage collection has run.")
+        public long getCollectionCount() {
+            return collectionCount;
+        }
+
+        public void setCollectionCount(long collectionCount) {
+            this.collectionCount = collectionCount;
+        }
+
+        @ApiModelProperty("The total amount of time spent garbage collecting.")
+        public String getCollectionTime() {
+            return collectionTime;
+        }
+
+        public void setCollectionTime(String collectionTime) {
+            this.collectionTime = collectionTime;
+        }
+
+        @ApiModelProperty("The total number of milliseconds spent garbage collecting.")
+        public Long getCollectionMillis() {
+            return collectionMillis;
+        }
+
+        public void setCollectionMillis(Long collectionMillis) {
+            this.collectionMillis = collectionMillis;
+        }
+
+        @Override
+        public GarbageCollectionDTO clone() {
+            final GarbageCollectionDTO other = new GarbageCollectionDTO();
+            other.setName(getName());
+            other.setCollectionCount(getCollectionCount());
+            other.setCollectionTime(getCollectionTime());
+            other.setCollectionMillis(getCollectionMillis());
+            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/ClusterConnectionStatusDTO.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/ClusterConnectionStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java
deleted file mode 100644
index 34f900f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java
+++ /dev/null
@@ -1,94 +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.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a connection's status across the cluster.
- */
-@XmlType(name = "clusterConnectionStatus")
-public class ClusterConnectionStatusDTO {
-
-    private Collection<NodeConnectionStatusDTO> nodeConnectionStatus;
-    private Date statsLastRefreshed;
-    private String connectionId;
-    private String connectionName;
-
-    /**
-     * @return time the status were last refreshed
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    @ApiModelProperty(
-            value = "The time the status was last refreshed."
-    )
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * @return connection id
-     */
-    @ApiModelProperty(
-            value = "The id of the connection."
-    )
-    public String getConnectionId() {
-        return connectionId;
-    }
-
-    public void setConnectionId(String connectionId) {
-        this.connectionId = connectionId;
-    }
-
-    /**
-     * @return connection name
-     */
-    @ApiModelProperty(
-            value = "The name of the connection."
-    )
-    public String getConnectionName() {
-        return connectionName;
-    }
-
-    public void setConnectionName(String connectionName) {
-        this.connectionName = connectionName;
-    }
-
-    /**
-     * @return The collection of node connection status DTO
-     */
-    @ApiModelProperty(
-            value = "The connection status for each node."
-    )
-    public Collection<NodeConnectionStatusDTO> getNodeConnectionStatus() {
-        return nodeConnectionStatus;
-    }
-
-    public void setNodeConnectionStatus(Collection<NodeConnectionStatusDTO> nodeConnectionStatus) {
-        this.nodeConnectionStatus = nodeConnectionStatus;
-    }
-
-}

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

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/ClusterProcessorStatusDTO.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/ClusterProcessorStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java
deleted file mode 100644
index faedc57..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java
+++ /dev/null
@@ -1,127 +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.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a processor's status across the cluster.
- */
-@XmlType(name = "clusterProcessorStatus")
-public class ClusterProcessorStatusDTO {
-
-    private Collection<NodeProcessorStatusDTO> nodeProcessorStatus;
-    private Date statsLastRefreshed;
-    private String processorId;
-    private String processorName;
-    private String processorType;
-    private String processorRunStatus;
-
-    /**
-     * @return time the status were last refreshed
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    @ApiModelProperty(
-            value = "The time when the status was last refreshed."
-    )
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * @return processor id
-     */
-    @ApiModelProperty(
-            value = "The processor id."
-    )
-    public String getProcessorId() {
-        return processorId;
-    }
-
-    public void setProcessorId(String processorId) {
-        this.processorId = processorId;
-    }
-
-    /**
-     * @return processor name
-     */
-    @ApiModelProperty(
-            value = "The processor name."
-    )
-    public String getProcessorName() {
-        return processorName;
-    }
-
-    public void setProcessorName(String processorName) {
-        this.processorName = processorName;
-    }
-
-    /**
-     * @return processor type
-     */
-    @ApiModelProperty(
-            value = "The processor type."
-    )
-    public String getProcessorType() {
-        return processorType;
-    }
-
-    public void setProcessorType(String processorType) {
-        this.processorType = processorType;
-    }
-
-    /**
-     * @return processor run status
-     */
-    @ApiModelProperty(
-            value = "The processor state.",
-            allowableValues = "RUNNING, STOPPED, DISABLED, INVALID"
-    )
-    public String getProcessorRunStatus() {
-        return processorRunStatus;
-    }
-
-    public void setProcessorRunStatus(String runStatus) {
-        this.processorRunStatus = runStatus;
-    }
-
-    /**
-     * Collection of node processor status DTO.
-     *
-     * @return The collection of node processor status DTO
-     */
-    @ApiModelProperty(
-            value = "The processor status for each node."
-    )
-    public Collection<NodeProcessorStatusDTO> getNodeProcessorStatus() {
-        return nodeProcessorStatus;
-    }
-
-    public void setNodeProcessorStatus(Collection<NodeProcessorStatusDTO> nodeProcessorStatus) {
-        this.nodeProcessorStatus = nodeProcessorStatus;
-    }
-
-}

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/ClusterRemoteProcessGroupStatusDTO.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/ClusterRemoteProcessGroupStatusDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java
deleted file mode 100644
index 5fef96f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java
+++ /dev/null
@@ -1,94 +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.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a remote process group's status across the cluster.
- */
-@XmlType(name = "clusterRemoteProcessGroupStatus")
-public class ClusterRemoteProcessGroupStatusDTO {
-
-    private Collection<NodeRemoteProcessGroupStatusDTO> nodeRemoteProcessGroupStatus;
-    private Date statsLastRefreshed;
-    private String remoteProcessGroupId;
-    private String remoteProcessGroupName;
-
-    /**
-     * @return the time the status were last refreshed
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    @ApiModelProperty(
-            value = "The time when the remote process group status was last refreshed."
-    )
-    public Date getStatsLastRefreshed() {
-        return statsLastRefreshed;
-    }
-
-    public void setStatsLastRefreshed(Date statsLastRefreshed) {
-        this.statsLastRefreshed = statsLastRefreshed;
-    }
-
-    /**
-     * @return remote process group status from each node in the cluster
-     */
-    @ApiModelProperty(
-            value = "The remote process group status from each node in the cluster."
-    )
-    public Collection<NodeRemoteProcessGroupStatusDTO> getNodeRemoteProcessGroupStatus() {
-        return nodeRemoteProcessGroupStatus;
-    }
-
-    public void setNodeRemoteProcessGroupStatus(Collection<NodeRemoteProcessGroupStatusDTO> nodeRemoteProcessGroupStatus) {
-        this.nodeRemoteProcessGroupStatus = nodeRemoteProcessGroupStatus;
-    }
-
-    /**
-     * @return remote process group id
-     */
-    @ApiModelProperty(
-            value = "The id of the remote process group."
-    )
-    public String getRemoteProcessGroupId() {
-        return remoteProcessGroupId;
-    }
-
-    public void setRemoteProcessGroupId(String remoteProcessGroupId) {
-        this.remoteProcessGroupId = remoteProcessGroupId;
-    }
-
-    /**
-     * @return remote process group name
-     */
-    @ApiModelProperty(
-            value = "The name of the remote process group."
-    )
-    public String getRemoteProcessGroupName() {
-        return remoteProcessGroupName;
-    }
-
-    public void setRemoteProcessGroupName(String remoteProcessGroupName) {
-        this.remoteProcessGroupName = remoteProcessGroupName;
-    }
-
-}

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/ClusterStatusHistoryDTO.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/ClusterStatusHistoryDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java
deleted file mode 100644
index 997e24b..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java
+++ /dev/null
@@ -1,79 +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.Collection;
-import java.util.Date;
-import javax.xml.bind.annotation.XmlType;
-import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter;
-import org.apache.nifi.web.api.dto.util.TimeAdapter;
-
-/**
- * DTO for serializing the a status history across the cluster.
- */
-@XmlType(name = "clusterStatusHistory")
-public class ClusterStatusHistoryDTO {
-
-    private Collection<NodeStatusHistoryDTO> nodeStatusHistory;
-    private StatusHistoryDTO clusterStatusHistory;
-    private Date generated;
-
-    /**
-     * @return when this status history was generated
-     */
-    @XmlJavaTypeAdapter(TimeAdapter.class)
-    @ApiModelProperty(
-            value = "When the status history was generated."
-    )
-    public Date getGenerated() {
-        return generated;
-    }
-
-    public void setGenerated(Date generated) {
-        this.generated = generated;
-    }
-
-    /**
-     * @return status history from each node in the cluster
-     */
-    @ApiModelProperty(
-            value = "The status history from each node."
-    )
-    public Collection<NodeStatusHistoryDTO> getNodeStatusHistory() {
-        return nodeStatusHistory;
-    }
-
-    public void setNodeStatusHistory(Collection<NodeStatusHistoryDTO> nodeStatusHistory) {
-        this.nodeStatusHistory = nodeStatusHistory;
-    }
-
-    /**
-     * @return status history for this component across the entire cluster
-     */
-    @ApiModelProperty(
-            value = "The status history for the entire cluster."
-    )
-    public StatusHistoryDTO getClusterStatusHistory() {
-        return clusterStatusHistory;
-    }
-
-    public void setClusterStatusHistory(StatusHistoryDTO clusterStatusHistory) {
-        this.clusterStatusHistory = clusterStatusHistory;
-    }
-
-}