You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by al...@apache.org on 2016/07/04 16:55:20 UTC

[6/6] nifi-minifi git commit: MINIFI-38 Removing reliance on JettyServer in order to add a flow status reporting end point. This also removes the UI and adds a 'flowstatus' option to minifi.sh to get information on the current flow from the terminal.

MINIFI-38 Removing reliance on JettyServer in order to add a flow status reporting end point. This also removes the UI and adds a 'flowstatus' option to minifi.sh to get information on the current flow from the terminal.

This closes #23.

Signed-off-by: Aldrin Piri <al...@apache.org>


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

Branch: refs/heads/master
Commit: 998205199fe1aae8396bbd18952840ae9e70c4db
Parents: 0689ac9
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Sun Jul 3 01:35:15 2016 -0400
Committer: Aldrin Piri <al...@apache.org>
Committed: Mon Jul 4 12:54:35 2016 -0400

----------------------------------------------------------------------
 minifi-assembly/pom.xml                         |  43 +-
 minifi-bootstrap/pom.xml                        |  16 +-
 .../apache/nifi/minifi/bootstrap/RunMiNiFi.java |  54 ++
 .../bootstrap/util/ConfigTransformer.java       |   9 +-
 .../util/ConfigTransformerException.java        |  32 +
 .../minifi/commons/status/FlowStatusReport.java | 152 ++++
 .../commons/status/common/BulletinStatus.java   |  71 ++
 .../commons/status/common/ValidationError.java  |  81 ++
 .../status/connection/ConnectionHealth.java     |  69 ++
 .../status/connection/ConnectionStats.java      |  93 ++
 .../status/connection/ConnectionStatusBean.java |  81 ++
 .../ControllerServiceHealth.java                |  85 ++
 .../ControllerServiceStatus.java                |  85 ++
 .../commons/status/instance/InstanceHealth.java |  97 +++
 .../commons/status/instance/InstanceStats.java  | 141 +++
 .../commons/status/instance/InstanceStatus.java |  86 ++
 .../status/processor/ProcessorHealth.java       |  85 ++
 .../status/processor/ProcessorStats.java        | 130 +++
 .../status/processor/ProcessorStatusBean.java   |  98 +++
 .../reportingTask/ReportingTaskHealth.java      |  97 +++
 .../reportingTask/ReportingTaskStatus.java      |  85 ++
 .../commons/status/rpg/InputPortStatus.java     |  81 ++
 .../status/rpg/RemoteProcessGroupHealth.java    | 105 +++
 .../status/rpg/RemoteProcessGroupStats.java     |  81 ++
 .../rpg/RemoteProcessGroupStatusBean.java       | 121 +++
 .../status/system/ContentRepositoryUsage.java   | 107 +++
 .../status/system/FlowfileRepositoryUsage.java  |  94 ++
 .../status/system/GarbageCollectionStatus.java  |  82 ++
 .../commons/status/system/HeapStatus.java       | 166 ++++
 .../status/system/SystemDiagnosticsStatus.java  | 107 +++
 .../status/system/SystemProcessorStats.java     |  73 ++
 .../status/util/StatusReportPopulator.java      | 375 ++++++++
 .../minifi/commons/status/TestStatusReport.java |  88 ++
 minifi-docs/Properties_Guide.md                 | 485 -----------
 minifi-docs/System_Admin_Guide.md               | 620 +++++++++++++
 .../minifi-framework-nar/pom.xml                |  26 +-
 .../minifi-framework-core/pom.xml               |  66 ++
 .../org/apache/nifi/minifi/MiNiFiServer.java    | 112 +++
 .../minifi/status/StatusConfigReporter.java     | 249 ++++++
 .../minifi/status/StatusRequestException.java   |  38 +
 .../nifi/minifi/status/StatusRequestParser.java | 416 +++++++++
 .../minifi/status/TestStatusConfigReporter.java | 873 +++++++++++++++++++
 .../minifi-framework/minifi-nar-utils/pom.xml   |  42 +
 .../org/apache/nifi/nar/ExtensionManager.java   | 182 ++++
 .../org/apache/nifi/nar/ExtensionMapping.java   |  73 ++
 .../org/apache/nifi/nar/NarClassLoader.java     | 228 +++++
 .../org/apache/nifi/nar/NarClassLoaders.java    | 273 ++++++
 .../java/org/apache/nifi/nar/NarCloseable.java  |  44 +
 .../nifi/nar/NarThreadContextClassLoader.java   | 189 ++++
 .../java/org/apache/nifi/nar/NarUnpacker.java   | 396 +++++++++
 .../java/org/apache/nifi/util/FileUtils.java    | 268 ++++++
 .../org/apache/nifi/nar/NarUnpackerTest.java    | 214 +++++
 .../resources/NarUnpacker/conf/nifi.properties  |  79 ++
 .../resources/NarUnpacker/lib/dummy-one.nar     | Bin 0 -> 1749 bytes
 .../NarUnpacker/lib/minifi-framework-nar.nar    | Bin 0 -> 446 bytes
 .../resources/NarUnpacker/lib2/dummy-two.nar    | Bin 0 -> 1751 bytes
 .../src/main/resources/bin/minifi.sh            |   4 +-
 .../minifi-framework/minifi-runtime/pom.xml     |  13 +
 .../apache/nifi/minifi/BootstrapListener.java   |  28 +-
 .../java/org/apache/nifi/minifi/MiNiFi.java     |  40 +-
 .../minifi-framework/pom.xml                    |   2 +
 .../pom.xml                                     |   2 +-
 .../minifi-provenance-repository-nar/pom.xml    |  12 +-
 pom.xml                                         |  46 +
 64 files changed, 7642 insertions(+), 548 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-assembly/pom.xml b/minifi-assembly/pom.xml
index fa2dab3..a76ed5e 100644
--- a/minifi-assembly/pom.xml
+++ b/minifi-assembly/pom.xml
@@ -100,10 +100,19 @@ limitations under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.minifi</groupId>
             <artifactId>minifi-runtime</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-framework-nar</artifactId>
+            <type>nar</type>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi.minifi</groupId>
             <artifactId>minifi-provenance-reporting-nar</artifactId>
             <type>nar</type>
         </dependency>
@@ -126,6 +135,14 @@ limitations under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-core-api</artifactId>
+        </dependency>
+        <!-- <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-core</artifactId>
+        </dependency>-->
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-docs</artifactId>
             <classifier>resources</classifier>
             <scope>runtime</scope>
@@ -133,42 +150,44 @@ limitations under the License.
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-framework-nar</artifactId>
+            <artifactId>nifi-standard-services-api-nar</artifactId>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-provenance-repository-nar</artifactId>
+            <artifactId>nifi-ssl-context-service-nar</artifactId>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-services-api-nar</artifactId>
+            <artifactId>nifi-standard-nar</artifactId>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-ssl-context-service-nar</artifactId>
+            <artifactId>nifi-update-attribute-nar</artifactId>
             <type>nar</type>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-standard-nar</artifactId>
-            <type>nar</type>
+            <artifactId>nifi-utils</artifactId>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-jetty-bundle</artifactId>
-            <type>nar</type>
+            <groupId>javax.servlet</groupId>
+            <artifactId>javax.servlet-api</artifactId>
+            <version>3.1.0</version>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-server</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-update-attribute-nar</artifactId>
-            <type>nar</type>
+            <artifactId>nifi-expression-language</artifactId>
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-utils</artifactId>
+            <artifactId>nifi-persistent-provenance-repository</artifactId>
         </dependency>
     </dependencies>
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-bootstrap/pom.xml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/pom.xml b/minifi-bootstrap/pom.xml
index 114396b..37a070b 100644
--- a/minifi-bootstrap/pom.xml
+++ b/minifi-bootstrap/pom.xml
@@ -50,6 +50,10 @@ limitations under the License.
             <artifactId>minifi-api</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi.minifi</groupId>
+            <artifactId>minifi-utils</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.yaml</groupId>
             <artifactId>snakeyaml</artifactId>
             <version>1.17</version>
@@ -75,18 +79,6 @@ limitations under the License.
             <artifactId>commons-io</artifactId>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.nifi</groupId>
-            <artifactId>nifi-framework-core</artifactId>
-            <version>0.6.0</version>
-            <exclusions>
-                <!-- excluding for IDE purposes, this jar doesn't wind up in the lib/bootstrap folder so isn't safe to use in bootstrap project -->
-                <exclusion>
-                    <groupId>org.apache.nifi</groupId>
-                    <artifactId>nifi-properties</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
     </dependencies>
 
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
index 82b583f..84156ec 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/RunMiNiFi.java
@@ -25,6 +25,7 @@ import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
 import java.io.OutputStream;
 import java.io.Reader;
 import java.lang.reflect.Field;
@@ -61,6 +62,7 @@ import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeExcepti
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
 import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
+import org.apache.nifi.minifi.commons.status.FlowStatusReport;
 import org.apache.nifi.stream.io.ByteArrayInputStream;
 import org.apache.nifi.stream.io.ByteArrayOutputStream;
 import org.apache.nifi.util.Tuple;
@@ -107,6 +109,7 @@ public class RunMiNiFi {
     public static final String RELOAD_CMD = "RELOAD";
     public static final String PING_CMD = "PING";
     public static final String DUMP_CMD = "DUMP";
+    public static final String FLOW_STATUS_REPORT_CMD = "FLOW_STATUS_REPORT";
 
     public static final String NOTIFIER_PROPERTY_PREFIX = "nifi.minifi.notifier";
     public static final String NOTIFIER_COMPONENTS_KEY = NOTIFIER_PROPERTY_PREFIX + ".components";
@@ -168,6 +171,7 @@ public class RunMiNiFi {
         System.out.println("Status : Determine if there is a running instance of Apache MiNiFi");
         System.out.println("Dump : Write a Thread Dump to the file specified by [options], or to the log if no file is given");
         System.out.println("Run : Start a new instance of Apache MiNiFi and monitor the Process, restarting if the instance dies");
+        System.out.println("FlowStatus : Get the status of the MiNiFi flow. For usage, read the System Admin Guide 'FlowStatus Query Options' section.");
         System.out.println();
     }
 
@@ -196,6 +200,7 @@ public class RunMiNiFi {
             case "dump":
             case "restart":
             case "env":
+            case "flowstatus":
                 break;
             default:
                 printUsage();
@@ -228,6 +233,13 @@ public class RunMiNiFi {
             case "env":
                 runMiNiFi.env();
                 break;
+            case "flowstatus":
+                if(args.length == 2) {
+                    runMiNiFi.statusReport(args[1]);
+                } else {
+                    System.out.println("The 'flowStatus' command requires input. See the System Admin Guide 'FlowStatus Query Options' section for complete details.");
+                }
+                break;
         }
     }
 
@@ -498,6 +510,19 @@ public class RunMiNiFi {
         }
     }
 
+    public void statusReport(String statusRequest) throws IOException {
+        final Logger logger = cmdLogger;
+        final Status status = getStatus(logger);
+        final Properties props = loadProperties(logger);
+
+        try {
+            FlowStatusReport flowStatusReport = getFlowStatusReport(statusRequest,status.getPort(), props.getProperty("secret.key"), logger);
+            System.out.println(flowStatusReport.toString());
+        } catch (IOException | ClassNotFoundException e) {
+            logger.error("Failed to get Flow Status", e);
+        }
+    }
+
     public void env() {
         final Logger logger = cmdLogger;
         final Status status = getStatus(logger);
@@ -1186,6 +1211,35 @@ public class RunMiNiFi {
         }
     }
 
+    public FlowStatusReport getFlowStatusReport(String statusRequest, final int port, final String secretKey, final Logger logger) throws IOException, ClassNotFoundException {
+        logger.debug("Pinging {}", port);
+
+        try (final Socket socket = new Socket("localhost", port)) {
+            final OutputStream out = socket.getOutputStream();
+            final String commandWithArgs = FLOW_STATUS_REPORT_CMD + " " + secretKey +" " + statusRequest + "\n";
+            out.write((commandWithArgs).getBytes(StandardCharsets.UTF_8));
+            logger.debug("Sending command to MiNiFi: {}",commandWithArgs);
+            out.flush();
+
+            logger.debug("Sent FLOW_STATUS_REPORT_CMD to MiNiFi");
+            socket.setSoTimeout(5000);
+            final InputStream in = socket.getInputStream();
+
+            ObjectInputStream ois = new ObjectInputStream(in);
+            logger.debug("FLOW_STATUS_REPORT_CMD response received");
+            Object o = ois.readObject();
+            ois.close();
+            out.close();
+            try {
+                return FlowStatusReport.class.cast(o);
+            } catch (ClassCastException e) {
+                String message = String.class.cast(o);
+                throw new IOException("Failed to get status report from MiNiFi due to:" + message);
+            }
+
+        }
+    }
+
     private void handleLogging(final Process process) {
         final Set<Future<?>> existingFutures = loggingFutures;
         if (existingFutures != null) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
index 1e44174..1a3b1fa 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
@@ -18,7 +18,6 @@
 package org.apache.nifi.minifi.bootstrap.util;
 
 
-import org.apache.nifi.controller.FlowSerializationException;
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
 import org.apache.nifi.minifi.bootstrap.exception.InvalidConfigurationException;
 import org.apache.nifi.minifi.bootstrap.util.schema.ComponentStatusRepositorySchema;
@@ -281,7 +280,7 @@ public final class ConfigTransformer {
         }
     }
 
-    private static DOMSource createFlowXml(ConfigSchema configSchema) throws IOException, ConfigurationChangeException {
+    private static DOMSource createFlowXml(ConfigSchema configSchema) throws IOException, ConfigurationChangeException, ConfigTransformerException{
         try {
             // create a new, empty document
             final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance();
@@ -314,9 +313,9 @@ public final class ConfigTransformer {
 
             return new DOMSource(doc);
         } catch (final ParserConfigurationException | DOMException | TransformerFactoryConfigurationError | IllegalArgumentException e) {
-            throw new FlowSerializationException(e);
-        } catch (Exception e) {
-            throw new ConfigurationChangeException("Failed to parse the config YAML while writing the top level of the flow xml", e);
+            throw new ConfigTransformerException(e);
+        } catch (Exception e){
+            throw new ConfigTransformerException("Failed to parse the config YAML while writing the top level of the flow xml", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformerException.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformerException.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformerException.java
new file mode 100644
index 0000000..b56b69b
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformerException.java
@@ -0,0 +1,32 @@
+/*
+ * 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.minifi.bootstrap.util;
+
+public class ConfigTransformerException extends Exception {
+    public ConfigTransformerException(String message) {
+        super(message);
+    }
+
+    public ConfigTransformerException(Throwable cause) {
+        super(cause);
+    }
+
+    public ConfigTransformerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/FlowStatusReport.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/FlowStatusReport.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/FlowStatusReport.java
new file mode 100644
index 0000000..d2b6851
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/FlowStatusReport.java
@@ -0,0 +1,152 @@
+/*
+ * 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.minifi.commons.status;
+
+import org.apache.nifi.minifi.commons.status.connection.ConnectionStatusBean;
+import org.apache.nifi.minifi.commons.status.controllerservice.ControllerServiceStatus;
+import org.apache.nifi.minifi.commons.status.instance.InstanceStatus;
+import org.apache.nifi.minifi.commons.status.processor.ProcessorStatusBean;
+import org.apache.nifi.minifi.commons.status.reportingTask.ReportingTaskStatus;
+import org.apache.nifi.minifi.commons.status.rpg.RemoteProcessGroupStatusBean;
+import org.apache.nifi.minifi.commons.status.system.SystemDiagnosticsStatus;
+
+import java.util.List;
+
+public class FlowStatusReport implements java.io.Serializable {
+    private List<ControllerServiceStatus> controllerServiceStatusList;
+    private List<ProcessorStatusBean> processorStatusList;
+    private List<ConnectionStatusBean> connectionStatusList;
+    private List<RemoteProcessGroupStatusBean> remoteProcessGroupStatusList;
+    private InstanceStatus instanceStatus;
+    private SystemDiagnosticsStatus systemDiagnosticsStatus;
+    private List<ReportingTaskStatus> reportingTaskStatusList;
+    private List<String> errorsGeneratingReport;
+
+    public FlowStatusReport() {
+    }
+
+    public List<ControllerServiceStatus> getControllerServiceStatusList() {
+        return controllerServiceStatusList;
+    }
+
+    public void setControllerServiceStatusList(List<ControllerServiceStatus> controllerServiceStatusList) {
+        this.controllerServiceStatusList = controllerServiceStatusList;
+    }
+
+    public List<ProcessorStatusBean> getProcessorStatusList() {
+        return processorStatusList;
+    }
+
+    public void setProcessorStatusList(List<ProcessorStatusBean> processorStatusList) {
+        this.processorStatusList = processorStatusList;
+    }
+
+    public List<ConnectionStatusBean> getConnectionStatusList() {
+        return connectionStatusList;
+    }
+
+    public void setConnectionStatusList(List<ConnectionStatusBean> connectionStatusList) {
+        this.connectionStatusList = connectionStatusList;
+    }
+
+    public InstanceStatus getInstanceStatus() {
+        return instanceStatus;
+    }
+
+    public void setInstanceStatus(InstanceStatus instanceStatus) {
+        this.instanceStatus = instanceStatus;
+    }
+
+    public SystemDiagnosticsStatus getSystemDiagnosticsStatus() {
+        return systemDiagnosticsStatus;
+    }
+
+    public void setSystemDiagnosticsStatus(SystemDiagnosticsStatus systemDiagnosticsStatus) {
+        this.systemDiagnosticsStatus = systemDiagnosticsStatus;
+    }
+
+    public List<RemoteProcessGroupStatusBean> getRemoteProcessGroupStatusList() {
+        return remoteProcessGroupStatusList;
+    }
+
+    public void setRemoteProcessGroupStatusList(List<RemoteProcessGroupStatusBean> remoteProcessGroupStatusList) {
+        this.remoteProcessGroupStatusList = remoteProcessGroupStatusList;
+    }
+
+    public List<ReportingTaskStatus> getReportingTaskStatusList() {
+        return reportingTaskStatusList;
+    }
+
+    public void setReportingTaskStatusList(List<ReportingTaskStatus> reportingTaskStatusList) {
+        this.reportingTaskStatusList = reportingTaskStatusList;
+    }
+
+    public List<String> getErrorsGeneratingReport() {
+        return errorsGeneratingReport;
+    }
+
+    public void setErrorsGeneratingReport(List<String> errorsGeneratingReport) {
+        this.errorsGeneratingReport = errorsGeneratingReport;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        FlowStatusReport that = (FlowStatusReport) o;
+
+        if (getControllerServiceStatusList() != null ? !getControllerServiceStatusList().equals(that.getControllerServiceStatusList()) : that.getControllerServiceStatusList() != null) return false;
+        if (getProcessorStatusList() != null ? !getProcessorStatusList().equals(that.getProcessorStatusList()) : that.getProcessorStatusList() != null) return false;
+        if (getConnectionStatusList() != null ? !getConnectionStatusList().equals(that.getConnectionStatusList()) : that.getConnectionStatusList() != null) return false;
+        if (getRemoteProcessGroupStatusList() != null ? !getRemoteProcessGroupStatusList().equals(that.getRemoteProcessGroupStatusList()) : that.getRemoteProcessGroupStatusList() != null)
+            return false;
+        if (getInstanceStatus() != null ? !getInstanceStatus().equals(that.getInstanceStatus()) : that.getInstanceStatus() != null) return false;
+        if (getSystemDiagnosticsStatus() != null ? !getSystemDiagnosticsStatus().equals(that.getSystemDiagnosticsStatus()) : that.getSystemDiagnosticsStatus() != null) return false;
+        if (getReportingTaskStatusList() != null ? !getReportingTaskStatusList().equals(that.getReportingTaskStatusList()) : that.getReportingTaskStatusList() != null) return false;
+        return getErrorsGeneratingReport() != null ? getErrorsGeneratingReport().equals(that.getErrorsGeneratingReport()) : that.getErrorsGeneratingReport() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getControllerServiceStatusList() != null ? getControllerServiceStatusList().hashCode() : 0;
+        result = 31 * result + (getProcessorStatusList() != null ? getProcessorStatusList().hashCode() : 0);
+        result = 31 * result + (getConnectionStatusList() != null ? getConnectionStatusList().hashCode() : 0);
+        result = 31 * result + (getRemoteProcessGroupStatusList() != null ? getRemoteProcessGroupStatusList().hashCode() : 0);
+        result = 31 * result + (getInstanceStatus() != null ? getInstanceStatus().hashCode() : 0);
+        result = 31 * result + (getSystemDiagnosticsStatus() != null ? getSystemDiagnosticsStatus().hashCode() : 0);
+        result = 31 * result + (getReportingTaskStatusList() != null ? getReportingTaskStatusList().hashCode() : 0);
+        result = 31 * result + (getErrorsGeneratingReport() != null ? getErrorsGeneratingReport().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "FlowStatusReport{" +
+                "controllerServiceStatusList=" + controllerServiceStatusList +
+                ", processorStatusList=" + processorStatusList +
+                ", connectionStatusList=" + connectionStatusList +
+                ", remoteProcessGroupStatusList=" + remoteProcessGroupStatusList +
+                ", instanceStatus=" + instanceStatus +
+                ", systemDiagnosticsStatus=" + systemDiagnosticsStatus +
+                ", reportingTaskStatusList=" + reportingTaskStatusList +
+                ", errorsGeneratingReport=" + errorsGeneratingReport +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/BulletinStatus.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/BulletinStatus.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/BulletinStatus.java
new file mode 100644
index 0000000..7d33ddc
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/BulletinStatus.java
@@ -0,0 +1,71 @@
+/*
+ * 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.minifi.commons.status.common;
+
+import java.util.Date;
+
+public class BulletinStatus implements java.io.Serializable {
+    private Date timestamp;
+    private String message;
+
+    public BulletinStatus() {
+    }
+
+    public Date getTimestamp() {
+        return timestamp;
+    }
+
+    public void setTimestamp(Date timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+
+    public void setMessage(String message) {
+        this.message = message;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        BulletinStatus bulletin = (BulletinStatus) o;
+
+        if (getTimestamp() != null ? !getTimestamp().equals(bulletin.getTimestamp()) : bulletin.getTimestamp() != null) return false;
+        return getMessage() != null ? getMessage().equals(bulletin.getMessage()) : bulletin.getMessage() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getTimestamp() != null ? getTimestamp().hashCode() : 0;
+        result = 31 * result + (getMessage() != null ? getMessage().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "timestamp=" + timestamp +
+                ", message='" + message + '\'' +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/ValidationError.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/ValidationError.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/ValidationError.java
new file mode 100644
index 0000000..6ee5697
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/common/ValidationError.java
@@ -0,0 +1,81 @@
+/*
+ * 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.minifi.commons.status.common;
+
+public class ValidationError implements java.io.Serializable {
+    private String subject;
+    private String input;
+    private String reason;
+
+    public ValidationError() {
+    }
+
+    public String getSubject() {
+        return subject;
+    }
+
+    public void setSubject(String subject) {
+        this.subject = subject;
+    }
+
+    public String getInput() {
+        return input;
+    }
+
+    public void setInput(String input) {
+        this.input = input;
+    }
+
+    public String getReason() {
+        return reason;
+    }
+
+    public void setReason(String reason) {
+        this.reason = reason;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ValidationError that = (ValidationError) o;
+
+        if (getSubject() != null ? !getSubject().equals(that.getSubject()) : that.getSubject() != null) return false;
+        if (getInput() != null ? !getInput().equals(that.getInput()) : that.getInput() != null) return false;
+        return getReason() != null ? getReason().equals(that.getReason()) : that.getReason() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getSubject() != null ? getSubject().hashCode() : 0;
+        result = 31 * result + (getInput() != null ? getInput().hashCode() : 0);
+        result = 31 * result + (getReason() != null ? getReason().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "subject='" + subject + '\'' +
+                ", input='" + input + '\'' +
+                ", reason='" + reason + '\'' +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionHealth.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionHealth.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionHealth.java
new file mode 100644
index 0000000..b53ca8c
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionHealth.java
@@ -0,0 +1,69 @@
+/*
+ * 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.minifi.commons.status.connection;
+
+public class ConnectionHealth implements java.io.Serializable {
+    private int queuedCount;
+    private long queuedBytes;
+
+    public ConnectionHealth() {
+    }
+
+    public int getQueuedCount() {
+        return queuedCount;
+    }
+
+    public void setQueuedCount(int queuedCount) {
+        this.queuedCount = queuedCount;
+    }
+
+    public long getQueuedBytes() {
+        return queuedBytes;
+    }
+
+    public void setQueuedBytes(long queuedBytes) {
+        this.queuedBytes = queuedBytes;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ConnectionHealth that = (ConnectionHealth) o;
+
+        if (getQueuedCount() != that.getQueuedCount()) return false;
+        return getQueuedBytes() == that.getQueuedBytes();
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getQueuedCount();
+        result = 31 * result + (int) (getQueuedBytes() ^ (getQueuedBytes() >>> 32));
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "queuedCount=" + queuedCount +
+                ", queuedBytes=" + queuedBytes +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStats.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStats.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStats.java
new file mode 100644
index 0000000..add7d85
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStats.java
@@ -0,0 +1,93 @@
+/*
+ * 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.minifi.commons.status.connection;
+
+public class ConnectionStats implements java.io.Serializable {
+    private int inputCount;
+    private long inputBytes;
+    private int outputCount;
+    private long outputBytes;
+
+    public ConnectionStats() {
+    }
+
+    public int getInputCount() {
+        return inputCount;
+    }
+
+    public void setInputCount(int inputCount) {
+        this.inputCount = inputCount;
+    }
+
+    public long getInputBytes() {
+        return inputBytes;
+    }
+
+    public void setInputBytes(long inputBytes) {
+        this.inputBytes = inputBytes;
+    }
+
+    public int getOutputCount() {
+        return outputCount;
+    }
+
+    public void setOutputCount(int outputCount) {
+        this.outputCount = outputCount;
+    }
+
+    public long getOutputBytes() {
+        return outputBytes;
+    }
+
+    public void setOutputBytes(long outputBytes) {
+        this.outputBytes = outputBytes;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ConnectionStats that = (ConnectionStats) o;
+
+        if (getInputCount() != that.getInputCount()) return false;
+        if (getInputBytes() != that.getInputBytes()) return false;
+        if (getOutputCount() != that.getOutputCount()) return false;
+        return getOutputBytes() == that.getOutputBytes();
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getInputCount();
+        result = 31 * result + (int) (getInputBytes() ^ (getInputBytes() >>> 32));
+        result = 31 * result + getOutputCount();
+        result = 31 * result + (int) (getOutputBytes() ^ (getOutputBytes() >>> 32));
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "inputCount=" + inputCount +
+                ", inputBytes=" + inputBytes +
+                ", outputCount=" + outputCount +
+                ", outputBytes=" + outputBytes +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStatusBean.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStatusBean.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStatusBean.java
new file mode 100644
index 0000000..378b95e
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/connection/ConnectionStatusBean.java
@@ -0,0 +1,81 @@
+/*
+ * 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.minifi.commons.status.connection;
+
+public class ConnectionStatusBean implements java.io.Serializable {
+    private String name;
+    private ConnectionHealth connectionHealth;
+    private ConnectionStats connectionStats;
+
+    public ConnectionStatusBean() {
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public ConnectionHealth getConnectionHealth() {
+        return connectionHealth;
+    }
+
+    public void setConnectionHealth(ConnectionHealth connectionHealth) {
+        this.connectionHealth = connectionHealth;
+    }
+
+    public ConnectionStats getConnectionStats() {
+        return connectionStats;
+    }
+
+    public void setConnectionStats(ConnectionStats connectionStats) {
+        this.connectionStats = connectionStats;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ConnectionStatusBean that = (ConnectionStatusBean) o;
+
+        if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) return false;
+        if (getConnectionHealth() != null ? !getConnectionHealth().equals(that.getConnectionHealth()) : that.getConnectionHealth() != null) return false;
+        return getConnectionStats() != null ? getConnectionStats().equals(that.getConnectionStats()) : that.getConnectionStats() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getName() != null ? getName().hashCode() : 0;
+        result = 31 * result + (getConnectionHealth() != null ? getConnectionHealth().hashCode() : 0);
+        result = 31 * result + (getConnectionStats() != null ? getConnectionStats().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "name='" + name + '\'' +
+                ", connectionHealth=" + connectionHealth +
+                ", connectionStats=" + connectionStats +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceHealth.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceHealth.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceHealth.java
new file mode 100644
index 0000000..8832c53
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceHealth.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.minifi.commons.status.controllerservice;
+
+import org.apache.nifi.minifi.commons.status.common.ValidationError;
+
+import java.util.List;
+
+public class ControllerServiceHealth implements java.io.Serializable {
+    private String state;
+    private boolean hasBulletins;
+    private List<ValidationError> validationErrorList;
+
+    public ControllerServiceHealth() {
+    }
+
+    public String getState() {
+        return state;
+    }
+
+    public void setState(String state) {
+        this.state = state;
+    }
+
+    public boolean isHasBulletins() {
+        return hasBulletins;
+    }
+
+    public void setHasBulletins(boolean hasBulletins) {
+        this.hasBulletins = hasBulletins;
+    }
+
+    public List<ValidationError> getValidationErrorList() {
+        return validationErrorList;
+    }
+
+    public void setValidationErrorList(List<ValidationError> validationErrorList) {
+        this.validationErrorList = validationErrorList;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ControllerServiceHealth that = (ControllerServiceHealth) o;
+
+        if (isHasBulletins() != that.isHasBulletins()) return false;
+        if (getState() != null ? !getState().equals(that.getState()) : that.getState() != null) return false;
+        return getValidationErrorList() != null ? getValidationErrorList().equals(that.getValidationErrorList()) : that.getValidationErrorList() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getState() != null ? getState().hashCode() : 0;
+        result = 31 * result + (isHasBulletins() ? 1 : 0);
+        result = 31 * result + (getValidationErrorList() != null ? getValidationErrorList().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "state='" + state + '\'' +
+                ", hasBulletins=" + hasBulletins +
+                ", validationErrorList=" + validationErrorList +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceStatus.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceStatus.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceStatus.java
new file mode 100644
index 0000000..91e5c9a
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/controllerservice/ControllerServiceStatus.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.minifi.commons.status.controllerservice;
+
+import org.apache.nifi.minifi.commons.status.common.BulletinStatus;
+
+import java.util.List;
+
+public class ControllerServiceStatus implements java.io.Serializable {
+    private String name;
+    private ControllerServiceHealth controllerServiceHealth;
+    private List<BulletinStatus> bulletinList;
+
+    public ControllerServiceStatus() {
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public ControllerServiceHealth getControllerServiceHealth() {
+        return controllerServiceHealth;
+    }
+
+    public void setControllerServiceHealth(ControllerServiceHealth controllerServiceHealth) {
+        this.controllerServiceHealth = controllerServiceHealth;
+    }
+
+    public List<BulletinStatus> getBulletinList() {
+        return bulletinList;
+    }
+
+    public void setBulletinList(List<BulletinStatus> bulletinList) {
+        this.bulletinList = bulletinList;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ControllerServiceStatus that = (ControllerServiceStatus) o;
+
+        if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) return false;
+        if (getControllerServiceHealth() != null ? !getControllerServiceHealth().equals(that.getControllerServiceHealth()) : that.getControllerServiceHealth() != null) return false;
+        return getBulletinList() != null ? getBulletinList().equals(that.getBulletinList()) : that.getBulletinList() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getName() != null ? getName().hashCode() : 0;
+        result = 31 * result + (getControllerServiceHealth() != null ? getControllerServiceHealth().hashCode() : 0);
+        result = 31 * result + (getBulletinList() != null ? getBulletinList().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "name='" + name + '\'' +
+                ", controllerServiceHealth=" + controllerServiceHealth +
+                ", bulletinList=" + bulletinList +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceHealth.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceHealth.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceHealth.java
new file mode 100644
index 0000000..e53e8a3
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceHealth.java
@@ -0,0 +1,97 @@
+/*
+ * 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.minifi.commons.status.instance;
+
+public class InstanceHealth implements java.io.Serializable {
+
+    private int queuedCount;
+    private double queuedContentSize;
+    private boolean hasBulletins;
+    private int activeThreads;
+
+    public InstanceHealth() {
+    }
+
+    public int getQueuedCount() {
+        return queuedCount;
+    }
+
+    public void setQueuedCount(int queuedCount) {
+        this.queuedCount = queuedCount;
+    }
+
+    public double getQueuedContentSize() {
+        return queuedContentSize;
+    }
+
+    public void setQueuedContentSize(double queuedContentSize) {
+        this.queuedContentSize = queuedContentSize;
+    }
+
+    public boolean isHasBulletins() {
+        return hasBulletins;
+    }
+
+    public void setHasBulletins(boolean hasBulletins) {
+        this.hasBulletins = hasBulletins;
+    }
+
+    public int getActiveThreads() {
+        return activeThreads;
+    }
+
+    public void setActiveThreads(int activeThreads) {
+        this.activeThreads = activeThreads;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        InstanceHealth that = (InstanceHealth) o;
+
+        if (getQueuedCount() != that.getQueuedCount()) return false;
+        if (Double.compare(that.getQueuedContentSize(), getQueuedContentSize()) != 0) return false;
+        if (isHasBulletins() != that.isHasBulletins()) return false;
+        return activeThreads == that.activeThreads;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result;
+        long temp;
+        result = getQueuedCount();
+        temp = Double.doubleToLongBits(getQueuedContentSize());
+        result = 31 * result + (int) (temp ^ (temp >>> 32));
+        result = 31 * result + (isHasBulletins() ? 1 : 0);
+        result = 31 * result + activeThreads;
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "queuedCount=" + queuedCount +
+                ", queuedContentSize=" + queuedContentSize +
+                ", hasBulletins=" + hasBulletins +
+                ", activeThreads=" + activeThreads +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStats.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStats.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStats.java
new file mode 100644
index 0000000..1521c21
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStats.java
@@ -0,0 +1,141 @@
+/*
+ * 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.minifi.commons.status.instance;
+
+public class InstanceStats implements java.io.Serializable {
+    private long bytesRead;
+    private long bytesWritten;
+    private long bytesSent;
+    private int flowfilesSent;
+    private long bytesTransferred;
+    private int flowfilesTransferred;
+    private long bytesReceived;
+    private int flowfilesReceived;
+
+    public InstanceStats() {
+    }
+
+    public long getBytesRead() {
+        return bytesRead;
+    }
+
+    public void setBytesRead(long bytesRead) {
+        this.bytesRead = bytesRead;
+    }
+
+    public long getBytesWritten() {
+        return bytesWritten;
+    }
+
+    public void setBytesWritten(long bytesWritten) {
+        this.bytesWritten = bytesWritten;
+    }
+
+    public long getBytesSent() {
+        return bytesSent;
+    }
+
+    public void setBytesSent(long bytesSent) {
+        this.bytesSent = bytesSent;
+    }
+
+    public int getFlowfilesSent() {
+        return flowfilesSent;
+    }
+
+    public void setFlowfilesSent(int flowfilesSent) {
+        this.flowfilesSent = flowfilesSent;
+    }
+
+    public long getBytesTransferred() {
+        return bytesTransferred;
+    }
+
+    public void setBytesTransferred(long bytesTransferred) {
+        this.bytesTransferred = bytesTransferred;
+    }
+
+    public int getFlowfilesTransferred() {
+        return flowfilesTransferred;
+    }
+
+    public void setFlowfilesTransferred(int flowfilesTransferred) {
+        this.flowfilesTransferred = flowfilesTransferred;
+    }
+
+    public long getBytesReceived() {
+        return bytesReceived;
+    }
+
+    public void setBytesReceived(long bytesReceived) {
+        this.bytesReceived = bytesReceived;
+    }
+
+    public int getFlowfilesReceived() {
+        return flowfilesReceived;
+    }
+
+    public void setFlowfilesReceived(int flowfilesReceived) {
+        this.flowfilesReceived = flowfilesReceived;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        InstanceStats that = (InstanceStats) o;
+
+        if (getBytesRead() != that.getBytesRead()) return false;
+        if (getBytesWritten() != that.getBytesWritten()) return false;
+        if (getBytesSent() != that.getBytesSent()) return false;
+        if (getFlowfilesSent() != that.getFlowfilesSent()) return false;
+        if (getBytesTransferred() != that.getBytesTransferred()) return false;
+        if (getFlowfilesTransferred() != that.getFlowfilesTransferred()) return false;
+        if (getBytesReceived() != that.getBytesReceived()) return false;
+        return getFlowfilesReceived() == that.getFlowfilesReceived();
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = (int) (getBytesRead() ^ (getBytesRead() >>> 32));
+        result = 31 * result + (int) (getBytesWritten() ^ (getBytesWritten() >>> 32));
+        result = 31 * result + (int) (getBytesSent() ^ (getBytesSent() >>> 32));
+        result = 31 * result + getFlowfilesSent();
+        result = 31 * result + (int) (getBytesTransferred() ^ (getBytesTransferred() >>> 32));
+        result = 31 * result + getFlowfilesTransferred();
+        result = 31 * result + (int) (getBytesReceived() ^ (getBytesReceived() >>> 32));
+        result = 31 * result + getFlowfilesReceived();
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "bytesRead=" + bytesRead +
+                ", bytesWritten=" + bytesWritten +
+                ", bytesSent=" + bytesSent +
+                ", flowfilesSent=" + flowfilesSent +
+                ", bytesTransferred=" + bytesTransferred +
+                ", flowfilesTransferred=" + flowfilesTransferred +
+                ", bytesReceived=" + bytesReceived +
+                ", flowfilesReceived=" + flowfilesReceived +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStatus.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStatus.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStatus.java
new file mode 100644
index 0000000..f535090
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/instance/InstanceStatus.java
@@ -0,0 +1,86 @@
+/*
+ * 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.minifi.commons.status.instance;
+
+import org.apache.nifi.minifi.commons.status.common.BulletinStatus;
+
+import java.util.List;
+
+public class InstanceStatus implements java.io.Serializable {
+
+    private InstanceHealth instanceHealth;
+    private List<BulletinStatus> bulletinList;
+    private InstanceStats instanceStats;
+
+    public InstanceStatus() {
+    }
+
+    public InstanceHealth getInstanceHealth() {
+        return instanceHealth;
+    }
+
+    public void setInstanceHealth(InstanceHealth instanceHealth) {
+        this.instanceHealth = instanceHealth;
+    }
+
+    public List<BulletinStatus> getBulletinList() {
+        return bulletinList;
+    }
+
+    public void setBulletinList(List<BulletinStatus> bulletinList) {
+        this.bulletinList = bulletinList;
+    }
+
+    public InstanceStats getInstanceStats() {
+        return instanceStats;
+    }
+
+    public void setInstanceStats(InstanceStats instanceStats) {
+        this.instanceStats = instanceStats;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        InstanceStatus that = (InstanceStatus) o;
+
+        if (getInstanceHealth() != null ? !getInstanceHealth().equals(that.getInstanceHealth()) : that.getInstanceHealth() != null) return false;
+        if (getBulletinList() != null ? !getBulletinList().equals(that.getBulletinList()) : that.getBulletinList() != null) return false;
+        return getInstanceStats() != null ? getInstanceStats().equals(that.getInstanceStats()) : that.getInstanceStats() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getInstanceHealth() != null ? getInstanceHealth().hashCode() : 0;
+        result = 31 * result + (getBulletinList() != null ? getBulletinList().hashCode() : 0);
+        result = 31 * result + (getInstanceStats() != null ? getInstanceStats().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "instanceHealth=" + instanceHealth +
+                ", bulletinList=" + bulletinList +
+                ", instanceStats=" + instanceStats +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorHealth.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorHealth.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorHealth.java
new file mode 100644
index 0000000..ebabd19
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorHealth.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.minifi.commons.status.processor;
+
+import org.apache.nifi.minifi.commons.status.common.ValidationError;
+
+import java.util.List;
+
+public class ProcessorHealth implements java.io.Serializable {
+    private String runStatus;
+    private boolean hasBulletins;
+    private List<ValidationError> validationErrorList;
+
+    public ProcessorHealth() {
+    }
+
+    public boolean isHasBulletins() {
+        return hasBulletins;
+    }
+
+    public void setHasBulletins(boolean hasBulletins) {
+        this.hasBulletins = hasBulletins;
+    }
+
+    public String getRunStatus() {
+        return runStatus;
+    }
+
+    public void setRunStatus(String runStatus) {
+        this.runStatus = runStatus;
+    }
+
+    public List<ValidationError> getValidationErrorList() {
+        return validationErrorList;
+    }
+
+    public void setValidationErrorList(List<ValidationError> validationErrorList) {
+        this.validationErrorList = validationErrorList;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ProcessorHealth that = (ProcessorHealth) o;
+
+        if (isHasBulletins() != that.isHasBulletins()) return false;
+        if (getRunStatus() != null ? !getRunStatus().equals(that.getRunStatus()) : that.getRunStatus() != null) return false;
+        return getValidationErrorList() != null ? getValidationErrorList().equals(that.getValidationErrorList()) : that.getValidationErrorList() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getRunStatus() != null ? getRunStatus().hashCode() : 0;
+        result = 31 * result + (isHasBulletins() ? 1 : 0);
+        result = 31 * result + (getValidationErrorList() != null ? getValidationErrorList().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "runStatus='" + runStatus + '\'' +
+                ", hasBulletins=" + hasBulletins +
+                ", validationErrorList=" + validationErrorList +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStats.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStats.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStats.java
new file mode 100644
index 0000000..9176fca
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStats.java
@@ -0,0 +1,130 @@
+/*
+ * 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.minifi.commons.status.processor;
+
+public class ProcessorStats implements java.io.Serializable {
+
+    private int activeThreads;
+    private int flowfilesReceived;
+    private long bytesRead;
+    private long bytesWritten;
+    private int flowfilesSent;
+    private int invocations;
+    private long processingNanos;
+
+    public ProcessorStats() {
+    }
+
+    public int getActiveThreads() {
+        return activeThreads;
+    }
+
+    public void setActiveThreads(int activeThreads) {
+        this.activeThreads = activeThreads;
+    }
+
+    public int getFlowfilesReceived() {
+        return flowfilesReceived;
+    }
+
+    public void setFlowfilesReceived(int flowfilesReceived) {
+        this.flowfilesReceived = flowfilesReceived;
+    }
+
+    public long getBytesRead() {
+        return bytesRead;
+    }
+
+    public void setBytesRead(long bytesRead) {
+        this.bytesRead = bytesRead;
+    }
+
+    public long getBytesWritten() {
+        return bytesWritten;
+    }
+
+    public void setBytesWritten(long bytesWritten) {
+        this.bytesWritten = bytesWritten;
+    }
+
+    public int getFlowfilesSent() {
+        return flowfilesSent;
+    }
+
+    public void setFlowfilesSent(int flowfilesSent) {
+        this.flowfilesSent = flowfilesSent;
+    }
+
+    public int getInvocations() {
+        return invocations;
+    }
+
+    public void setInvocations(int invocations) {
+        this.invocations = invocations;
+    }
+
+    public long getProcessingNanos() {
+        return processingNanos;
+    }
+
+    public void setProcessingNanos(long processingNanos) {
+        this.processingNanos = processingNanos;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ProcessorStats that = (ProcessorStats) o;
+
+        if (getActiveThreads() != that.getActiveThreads()) return false;
+        if (getFlowfilesReceived() != that.getFlowfilesReceived()) return false;
+        if (getBytesRead() != that.getBytesRead()) return false;
+        if (getBytesWritten() != that.getBytesWritten()) return false;
+        if (getFlowfilesSent() != that.getFlowfilesSent()) return false;
+        if (getInvocations() != that.getInvocations()) return false;
+        return getProcessingNanos() == that.getProcessingNanos();
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getActiveThreads();
+        result = 31 * result + getFlowfilesReceived();
+        result = 31 * result + (int) (getBytesRead() ^ (getBytesRead() >>> 32));
+        result = 31 * result + (int) (getBytesWritten() ^ (getBytesWritten() >>> 32));
+        result = 31 * result + getFlowfilesSent();
+        result = 31 * result + getInvocations();
+        result = 31 * result + (int) (getProcessingNanos() ^ (getProcessingNanos() >>> 32));
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "activeThreads=" + activeThreads +
+                ", flowfilesReceived=" + flowfilesReceived +
+                ", bytesRead=" + bytesRead +
+                ", bytesWritten=" + bytesWritten +
+                ", flowfilesSent=" + flowfilesSent +
+                ", invocations=" + invocations +
+                ", processingNanos=" + processingNanos +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStatusBean.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStatusBean.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStatusBean.java
new file mode 100644
index 0000000..99b597a
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/processor/ProcessorStatusBean.java
@@ -0,0 +1,98 @@
+/*
+ * 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.minifi.commons.status.processor;
+
+import org.apache.nifi.minifi.commons.status.common.BulletinStatus;
+
+import java.util.List;
+
+public class ProcessorStatusBean implements java.io.Serializable {
+
+    private String name;
+    private ProcessorHealth processorHealth;
+    private ProcessorStats processorStats;
+    private List<BulletinStatus> bulletinList;
+
+    public ProcessorStatusBean() {
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public ProcessorHealth getProcessorHealth() {
+        return processorHealth;
+    }
+
+    public void setProcessorHealth(ProcessorHealth processorHealth) {
+        this.processorHealth = processorHealth;
+    }
+
+    public ProcessorStats getProcessorStats() {
+        return processorStats;
+    }
+
+    public void setProcessorStats(ProcessorStats processorStats) {
+        this.processorStats = processorStats;
+    }
+
+    public List<BulletinStatus> getBulletinList() {
+        return bulletinList;
+    }
+
+    public void setBulletinList(List<BulletinStatus> bulletinList) {
+        this.bulletinList = bulletinList;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ProcessorStatusBean that = (ProcessorStatusBean) o;
+
+        if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) return false;
+        if (getProcessorHealth() != null ? !getProcessorHealth().equals(that.getProcessorHealth()) : that.getProcessorHealth() != null) return false;
+        if (getProcessorStats() != null ? !getProcessorStats().equals(that.getProcessorStats()) : that.getProcessorStats() != null) return false;
+        return getBulletinList() != null ? getBulletinList().equals(that.getBulletinList()) : that.getBulletinList() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getName() != null ? getName().hashCode() : 0;
+        result = 31 * result + (getProcessorHealth() != null ? getProcessorHealth().hashCode() : 0);
+        result = 31 * result + (getProcessorStats() != null ? getProcessorStats().hashCode() : 0);
+        result = 31 * result + (getBulletinList() != null ? getBulletinList().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "name='" + name + '\'' +
+                ", processorHealth=" + processorHealth +
+                ", processorStats=" + processorStats +
+                ", bulletinList=" + bulletinList +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskHealth.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskHealth.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskHealth.java
new file mode 100644
index 0000000..af869b5
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskHealth.java
@@ -0,0 +1,97 @@
+/*
+ * 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.minifi.commons.status.reportingTask;
+
+import org.apache.nifi.minifi.commons.status.common.ValidationError;
+
+import java.util.List;
+
+public class ReportingTaskHealth implements java.io.Serializable {
+    private String scheduledState;
+    private boolean hasBulletins;
+    private int activeThreads;
+    private List<ValidationError> validationErrorList;
+
+    public ReportingTaskHealth() {
+    }
+
+    public String getScheduledState() {
+        return scheduledState;
+    }
+
+    public void setScheduledState(String scheduledState) {
+        this.scheduledState = scheduledState;
+    }
+
+    public boolean isHasBulletins() {
+        return hasBulletins;
+    }
+
+    public void setHasBulletins(boolean hasBulletins) {
+        this.hasBulletins = hasBulletins;
+    }
+
+    public int getActiveThreads() {
+        return activeThreads;
+    }
+
+    public void setActiveThreads(int activeThreads) {
+        this.activeThreads = activeThreads;
+    }
+
+    public List<ValidationError> getValidationErrorList() {
+        return validationErrorList;
+    }
+
+    public void setValidationErrorList(List<ValidationError> validationErrorList) {
+        this.validationErrorList = validationErrorList;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ReportingTaskHealth that = (ReportingTaskHealth) o;
+
+        if (isHasBulletins() != that.isHasBulletins()) return false;
+        if (getActiveThreads() != that.getActiveThreads()) return false;
+        if (getScheduledState() != null ? !getScheduledState().equals(that.getScheduledState()) : that.getScheduledState() != null) return false;
+        return getValidationErrorList() != null ? getValidationErrorList().equals(that.getValidationErrorList()) : that.getValidationErrorList() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getScheduledState() != null ? getScheduledState().hashCode() : 0;
+        result = 31 * result + (isHasBulletins() ? 1 : 0);
+        result = 31 * result + getActiveThreads();
+        result = 31 * result + (getValidationErrorList() != null ? getValidationErrorList().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "scheduledState='" + scheduledState + '\'' +
+                ", hasBulletins=" + hasBulletins +
+                ", activeThreads=" + activeThreads +
+                ", validationErrorList=" + validationErrorList +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskStatus.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskStatus.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskStatus.java
new file mode 100644
index 0000000..ebc41c6
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/reportingTask/ReportingTaskStatus.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.minifi.commons.status.reportingTask;
+
+import org.apache.nifi.minifi.commons.status.common.BulletinStatus;
+
+import java.util.List;
+
+public class ReportingTaskStatus implements java.io.Serializable {
+    private String name;
+    private ReportingTaskHealth reportingTaskHealth;
+    private List<BulletinStatus> bulletinList;
+
+    public ReportingTaskStatus() {
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public ReportingTaskHealth getReportingTaskHealth() {
+        return reportingTaskHealth;
+    }
+
+    public void setReportingTaskHealth(ReportingTaskHealth reportingTaskHealth) {
+        this.reportingTaskHealth = reportingTaskHealth;
+    }
+
+    public List<BulletinStatus> getBulletinList() {
+        return bulletinList;
+    }
+
+    public void setBulletinList(List<BulletinStatus> bulletinList) {
+        this.bulletinList = bulletinList;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ReportingTaskStatus that = (ReportingTaskStatus) o;
+
+        if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) return false;
+        if (getReportingTaskHealth() != null ? !getReportingTaskHealth().equals(that.getReportingTaskHealth()) : that.getReportingTaskHealth() != null) return false;
+        return getBulletinList() != null ? getBulletinList().equals(that.getBulletinList()) : that.getBulletinList() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getName() != null ? getName().hashCode() : 0;
+        result = 31 * result + (getReportingTaskHealth() != null ? getReportingTaskHealth().hashCode() : 0);
+        result = 31 * result + (getBulletinList() != null ? getBulletinList().hashCode() : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "name='" + name + '\'' +
+                ", reportingTaskHealth=" + reportingTaskHealth +
+                ", bulletinList=" + bulletinList +
+                '}';
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/99820519/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/rpg/InputPortStatus.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/rpg/InputPortStatus.java b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/rpg/InputPortStatus.java
new file mode 100644
index 0000000..634580e
--- /dev/null
+++ b/minifi-commons/minifi-utils/src/main/java/org/apache/nifi/minifi/commons/status/rpg/InputPortStatus.java
@@ -0,0 +1,81 @@
+/*
+ * 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.minifi.commons.status.rpg;
+
+public class InputPortStatus implements java.io.Serializable {
+    private String name;
+    private boolean targetExists;
+    private boolean targetRunning;
+
+    public InputPortStatus() {
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public boolean isTargetExists() {
+        return targetExists;
+    }
+
+    public void setTargetExists(boolean targetExists) {
+        this.targetExists = targetExists;
+    }
+
+    public boolean isTargetRunning() {
+        return targetRunning;
+    }
+
+    public void setTargetRunning(boolean targetRunning) {
+        this.targetRunning = targetRunning;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        InputPortStatus inputPortStatus = (InputPortStatus) o;
+
+        if (isTargetExists() != inputPortStatus.isTargetExists()) return false;
+        if (isTargetRunning() != inputPortStatus.isTargetRunning()) return false;
+        return getName() != null ? getName().equals(inputPortStatus.getName()) : inputPortStatus.getName() == null;
+
+    }
+
+    @Override
+    public int hashCode() {
+        int result = getName() != null ? getName().hashCode() : 0;
+        result = 31 * result + (isTargetExists() ? 1 : 0);
+        result = 31 * result + (isTargetRunning() ? 1 : 0);
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return "{" +
+                "name='" + name + '\'' +
+                ", targetExists=" + targetExists +
+                ", targetRunning=" + targetRunning +
+                '}';
+    }
+}