You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jp...@apache.org on 2016/10/20 19:31:03 UTC

[1/3] nifi-minifi git commit: MINIFI-39 Adding PeriodicStatusReporter to the bootstrap process with an implementation to log the FlowStatus

Repository: nifi-minifi
Updated Branches:
  refs/heads/master 09b35f3ab -> 4e1363a2e


MINIFI-39 Adding PeriodicStatusReporter to the bootstrap process with an implementation to log the FlowStatus


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

Branch: refs/heads/master
Commit: 291b171cab8a86f4272e87318be3ed96ada128d5
Parents: 09b35f3
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Tue Oct 18 16:33:25 2016 -0400
Committer: Joseph Percivall <jo...@yahoo.com>
Committed: Thu Oct 20 15:25:50 2016 -0400

----------------------------------------------------------------------
 .../nifi/minifi/bootstrap/BootstrapCodec.java   |   1 +
 .../bootstrap/QueryableStatusAggregator.java    |  27 +++
 .../apache/nifi/minifi/bootstrap/RunMiNiFi.java |  90 ++++++--
 .../nifi/minifi/bootstrap/ShutdownHook.java     |  11 +
 .../status/PeriodicStatusReporter.java          |  81 +++++++
 .../status/reporters/StatusLogger.java          | 135 ++++++++++++
 .../status/reporters/TestStatusLogger.java      | 209 +++++++++++++++++++
 .../src/main/markdown/System_Admin_Guide.md     |  95 +++++++--
 .../src/main/resources/conf/bootstrap.conf      |  12 ++
 9 files changed, 635 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java
index 8dc2b7b..95e6f87 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/BootstrapCodec.java
@@ -111,6 +111,7 @@ public class BootstrapCodec {
             case "SHUTDOWN": {
                 logger.debug("Received 'SHUTDOWN' command from MINIFI");
                 runner.shutdownChangeNotifiers();
+                runner.shutdownPeriodicStatusReporters();
                 writer.write("OK");
                 writer.newLine();
                 writer.flush();

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/QueryableStatusAggregator.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/QueryableStatusAggregator.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/QueryableStatusAggregator.java
new file mode 100644
index 0000000..958a4d5
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/QueryableStatusAggregator.java
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+import org.apache.nifi.minifi.commons.status.FlowStatusReport;
+
+import java.io.IOException;
+
+public interface QueryableStatusAggregator {
+
+    FlowStatusReport statusReport(String statusRequest) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/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 84156ec..ad54c61 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
@@ -17,6 +17,7 @@
 package org.apache.nifi.minifi.bootstrap;
 
 import java.io.BufferedReader;
+import java.io.EOFException;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
@@ -33,6 +34,7 @@ import java.lang.reflect.Method;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.net.SocketTimeoutException;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
@@ -43,6 +45,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -61,6 +64,7 @@ import java.util.concurrent.locks.ReentrantLock;
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeException;
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeListener;
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
+import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
 import org.apache.nifi.minifi.bootstrap.util.ConfigTransformer;
 import org.apache.nifi.minifi.commons.status.FlowStatusReport;
 import org.apache.nifi.stream.io.ByteArrayInputStream;
@@ -86,7 +90,7 @@ import static java.nio.file.StandardCopyOption.REPLACE_EXISTING;
  * <p>
  * If the {@code bootstrap.conf} file cannot be found, throws a {@code FileNotFoundException}.
  */
-public class RunMiNiFi {
+public class RunMiNiFi implements QueryableStatusAggregator {
 
     public static final String DEFAULT_CONFIG_FILE = "./conf/bootstrap.conf";
     public static final String DEFAULT_NIFI_PROPS_FILE = "./conf/nifi.properties";
@@ -114,6 +118,9 @@ public class RunMiNiFi {
     public static final String NOTIFIER_PROPERTY_PREFIX = "nifi.minifi.notifier";
     public static final String NOTIFIER_COMPONENTS_KEY = NOTIFIER_PROPERTY_PREFIX + ".components";
 
+    public static final String STATUS_REPORTER_PROPERTY_PREFIX = "nifi.minifi.status.reporter";
+    public static final String STATUS_REPORTER_COMPONENTS_KEY = STATUS_REPORTER_PROPERTY_PREFIX + ".components";
+
     private volatile boolean autoRestartNiFi = true;
     private volatile int ccPort = -1;
     private volatile long nifiPid = -1L;
@@ -137,6 +144,8 @@ public class RunMiNiFi {
     private volatile int gracefulShutdownSeconds;
 
     private Set<ConfigurationChangeNotifier> changeNotifiers;
+    private Set<PeriodicStatusReporter> periodicStatusReporters;
+
     private MiNiFiConfigurationChangeListener changeListener;
 
     // Is set to true after the MiNiFi instance shuts down in preparation to be reloaded. Will be set to false after MiNiFi is successfully started again.
@@ -235,9 +244,9 @@ public class RunMiNiFi {
                 break;
             case "flowstatus":
                 if(args.length == 2) {
-                    runMiNiFi.statusReport(args[1]);
+                    System.out.println(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.");
+                    System.out.println("The 'flowStatus' command requires an input query. See the System Admin Guide 'FlowStatus Script Query' section for complete details.");
                 }
                 break;
         }
@@ -510,17 +519,27 @@ public class RunMiNiFi {
         }
     }
 
-    public void statusReport(String statusRequest) throws IOException {
+    public FlowStatusReport 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);
+        List<String> problemsGeneratingReport = new LinkedList<>();
+        if (!status.isProcessRunning()) {
+            problemsGeneratingReport.add("MiNiFi process is not running");
+        }
+
+        if (!status.isRespondingToPing()) {
+            problemsGeneratingReport.add("MiNiFi process is not responding to pings");
+        }
+
+        if (!problemsGeneratingReport.isEmpty()) {
+            FlowStatusReport flowStatusReport = new FlowStatusReport();
+            flowStatusReport.setErrorsGeneratingReport(problemsGeneratingReport);
+            return flowStatusReport;
         }
+
+        return getFlowStatusReport(statusRequest, status.getPort(), props.getProperty("secret.key"), logger);
     }
 
     public void env() {
@@ -1095,6 +1114,8 @@ public class RunMiNiFi {
         // Instantiate configuration listener and configured notifiers
         this.changeListener = new MiNiFiConfigurationChangeListener(this, defaultLogger);
         this.changeNotifiers = initializeNotifiers(this.changeListener);
+        this.periodicStatusReporters = initializePeriodicNotifiers();
+        startPeriodicNotifiers();
 
         ProcessBuilder builder = tuple.getKey();
         Process process = tuple.getValue();
@@ -1208,10 +1229,11 @@ public class RunMiNiFi {
             }
         } finally {
             shutdownChangeNotifiers();
+            shutdownPeriodicStatusReporters();
         }
     }
 
-    public FlowStatusReport getFlowStatusReport(String statusRequest, final int port, final String secretKey, final Logger logger) throws IOException, ClassNotFoundException {
+    public FlowStatusReport getFlowStatusReport(String statusRequest, final int port, final String secretKey, final Logger logger) throws IOException {
         logger.debug("Pinging {}", port);
 
         try (final Socket socket = new Socket("localhost", port)) {
@@ -1234,9 +1256,12 @@ public class RunMiNiFi {
                 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);
+                FlowStatusReport flowStatusReport = new FlowStatusReport();
+                flowStatusReport.setErrorsGeneratingReport(Collections.singletonList("Failed to get status report from MiNiFi due to:" + message));
+                return flowStatusReport;
             }
-
+        } catch (EOFException | ClassNotFoundException | SocketTimeoutException e) {
+            throw new IllegalStateException("Failed to get the status report from the MiNiFi process. Potentially due to the process currently being down (restarting or otherwise).", e);
         }
     }
 
@@ -1436,6 +1461,47 @@ public class RunMiNiFi {
         return changeNotifiers;
     }
 
+    public Set<PeriodicStatusReporter> getPeriodicStatusReporters() {
+        return Collections.unmodifiableSet(periodicStatusReporters);
+    }
+
+    public void shutdownPeriodicStatusReporters() {
+        for (PeriodicStatusReporter periodicStatusReporter : getPeriodicStatusReporters()) {
+            try {
+                periodicStatusReporter.stop();
+            } catch (Exception exception) {
+                System.out.println("Could not successfully stop periodic status reporter " + periodicStatusReporter.getClass() + " due to " + exception);
+            }
+        }
+    }
+
+    private Set<PeriodicStatusReporter> initializePeriodicNotifiers() throws IOException {
+        final Set<PeriodicStatusReporter> statusReporters = new HashSet<>();
+
+        final Properties bootstrapProperties = getBootstrapProperties();
+
+        final String reportersCsv = bootstrapProperties.getProperty(STATUS_REPORTER_COMPONENTS_KEY);
+        if (reportersCsv != null && !reportersCsv.isEmpty()) {
+            for (String reporterClassname : Arrays.asList(reportersCsv.split(","))) {
+                try {
+                    Class<?> reporterClass = Class.forName(reporterClassname);
+                    PeriodicStatusReporter reporter = (PeriodicStatusReporter) reporterClass.newInstance();
+                    reporter.initialize(bootstrapProperties, this);
+                    statusReporters.add(reporter);
+                } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+                    throw new RuntimeException("Issue instantiating notifier " + reporterClassname, e);
+                }
+            }
+        }
+        return statusReporters;
+    }
+
+    private void startPeriodicNotifiers() throws IOException {
+        for (PeriodicStatusReporter periodicStatusReporter: this.periodicStatusReporters) {
+            periodicStatusReporter.start();
+        }
+    }
+
     private static class MiNiFiConfigurationChangeListener implements ConfigurationChangeListener {
 
         private final RunMiNiFi runner;

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java
index ad3a2df..bec39e6 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/ShutdownHook.java
@@ -26,6 +26,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.nifi.minifi.bootstrap.configuration.ConfigurationChangeNotifier;
+import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
 
 public class ShutdownHook extends Thread {
 
@@ -60,6 +61,16 @@ public class ShutdownHook extends Thread {
                 System.out.println("Could not successfully stop notifier " + notifier.getClass() + " due to " + ioe);
             }
         }
+
+        System.out.println("Initiating shutdown of bootstrap periodic status reporters...");
+        for (PeriodicStatusReporter periodicStatusReporter : runner.getPeriodicStatusReporters()) {
+            try {
+                periodicStatusReporter.stop();
+            } catch (Exception exception) {
+                System.out.println("Could not successfully stop periodic status reporter " + periodicStatusReporter.getClass() + " due to " + exception);
+            }
+        }
+
         runner.setAutoRestartNiFi(false);
         final int ccPort = runner.getNiFiCommandControlPort();
         if (ccPort > 0) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java
new file mode 100644
index 0000000..8f0059c
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.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.bootstrap.status;
+
+import org.apache.nifi.minifi.bootstrap.QueryableStatusAggregator;
+
+import java.util.Properties;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+public abstract class PeriodicStatusReporter {
+
+    private final ScheduledThreadPoolExecutor scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+
+    private volatile int period = -1;
+    private volatile int termination_wait = 5000;
+
+    public volatile Runnable reportRunner;
+
+    /**
+     * Provides an opportunity for the implementation to perform configuration and initialization based on properties received from the bootstrapping configuration.
+     *
+     * @param properties from the bootstrap configuration
+     */
+    public abstract void initialize(Properties properties, QueryableStatusAggregator queryableStatusAggregator);
+
+    /**
+     * Begins the associated reporting service provided by the given implementation.  In most implementations, no action will occur until this method is invoked. The implementing class must have set
+     * 'reportRunner' prior to this method being called.
+     *
+     */
+    public void start() {
+        if (reportRunner == null){
+            throw new IllegalStateException("Programmatic error, the reportRunner has is still NULL when 'start' was called.");
+        }
+        scheduledExecutorService.scheduleAtFixedRate(reportRunner, period, period, TimeUnit.MILLISECONDS);
+    }
+
+    /**
+     * Stops the associated reporting service provided by the given implementation.
+     */
+    public void stop() {
+        try {
+            scheduledExecutorService.shutdown();
+            scheduledExecutorService.awaitTermination(termination_wait, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException ignore) {
+            // Shutting down anyway
+        }
+    }
+
+    public int getPeriod() {
+        return period;
+    }
+
+    public void setPeriod(int period) {
+        this.period = period;
+    }
+
+    public int getTermination_wait() {
+        return termination_wait;
+    }
+
+    public void setTermination_wait(int termination_wait) {
+        this.termination_wait = termination_wait;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/reporters/StatusLogger.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/reporters/StatusLogger.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/reporters/StatusLogger.java
new file mode 100644
index 0000000..ec7e7fc
--- /dev/null
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/reporters/StatusLogger.java
@@ -0,0 +1,135 @@
+/*
+ * 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.status.reporters;
+
+import org.apache.nifi.logging.LogLevel;
+import org.apache.nifi.minifi.bootstrap.QueryableStatusAggregator;
+import org.apache.nifi.minifi.bootstrap.status.PeriodicStatusReporter;
+import org.apache.nifi.minifi.commons.status.FlowStatusReport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import static org.apache.nifi.minifi.bootstrap.RunMiNiFi.STATUS_REPORTER_PROPERTY_PREFIX;
+
+public class StatusLogger extends PeriodicStatusReporter {
+
+
+    private volatile QueryableStatusAggregator queryableStatusAggregator;
+    private volatile LogLevel logLevel;
+    private volatile String statusQuery;
+
+    private static final Logger logger = LoggerFactory.getLogger(StatusLogger.class);
+
+
+    public static final String LOGGER_STATUS_REPORTER_PROPERTY_PREFIX = STATUS_REPORTER_PROPERTY_PREFIX + ".log";
+    public static final String REPORT_PERIOD_KEY = LOGGER_STATUS_REPORTER_PROPERTY_PREFIX + ".period";
+    public static final String LOGGING_LEVEL_KEY = LOGGER_STATUS_REPORTER_PROPERTY_PREFIX + ".level";
+    public static final String QUERY_KEY = LOGGER_STATUS_REPORTER_PROPERTY_PREFIX + ".query";
+
+    static final String ENCOUNTERED_IO_EXCEPTION = "Encountered an IO Exception while attempting to query the flow status.";
+
+    @Override
+    public void initialize(Properties properties,  QueryableStatusAggregator queryableStatusAggregator) {
+        this.queryableStatusAggregator = queryableStatusAggregator;
+
+        String periodString = properties.getProperty(REPORT_PERIOD_KEY);
+        if (periodString == null) {
+            throw new IllegalStateException(REPORT_PERIOD_KEY + " is null but it is required. Please configure it.");
+        }
+        try {
+            setPeriod(Integer.parseInt(periodString));
+        } catch (NumberFormatException e) {
+            throw new IllegalStateException(REPORT_PERIOD_KEY + " is not a valid number.", e);
+        }
+
+
+        String loglevelString = properties.getProperty(LOGGING_LEVEL_KEY);
+        if (loglevelString == null) {
+            throw new IllegalStateException(LOGGING_LEVEL_KEY + " is null but it is required. Please configure it.");
+        }
+
+        try {
+            logLevel = LogLevel.valueOf(loglevelString.toUpperCase());
+        } catch (IllegalArgumentException e) {
+            throw new IllegalStateException("Value set for " + LOGGING_LEVEL_KEY + " is not a valid log level.");
+        }
+
+        if (LogLevel.FATAL.equals(logLevel)){
+            throw new IllegalStateException("Cannot log status at the FATAL level. Please configure " + LOGGING_LEVEL_KEY + " to another value.");
+        }
+
+        statusQuery = properties.getProperty(QUERY_KEY);
+        if (statusQuery == null) {
+            throw new IllegalStateException(QUERY_KEY + " is null but it is required. Please configure it.");
+        }
+
+        reportRunner = new ReportStatusRunner();
+    }
+
+
+    private class ReportStatusRunner implements Runnable {
+
+        @Override
+        public void run() {
+            try {
+                String toLog;
+                Exception exception = null;
+                try {
+                    FlowStatusReport flowStatusReport = queryableStatusAggregator.statusReport(statusQuery);
+                    toLog = flowStatusReport.toString();
+                } catch (IOException e) {
+                    toLog = ENCOUNTERED_IO_EXCEPTION;
+                    exception = e;
+                }
+
+                // If exception is null the logger will ignore it.
+                switch (logLevel) {
+                    case TRACE:
+                        logger.trace(toLog, exception);
+                        break;
+                    case DEBUG:
+                        logger.debug(toLog, exception);
+                        break;
+                    case INFO:
+                        logger.info(toLog, exception);
+                        break;
+                    case WARN:
+                        logger.warn(toLog, exception);
+                        break;
+                    case ERROR:
+                        logger.error(toLog, exception);
+                        break;
+                    default:
+                        throw new IllegalStateException("Cannot log status at level " + logLevel + ". Please configure another.");
+                }
+
+            } catch (Exception e) {
+                switch (logLevel) {
+                    case ERROR:
+                        logger.error("Unexpected exception when attempting to report the status", e);
+                        break;
+                    default:
+                        logger.warn("Unexpected exception when attempting to report the status", e);
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/status/reporters/TestStatusLogger.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/status/reporters/TestStatusLogger.java b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/status/reporters/TestStatusLogger.java
new file mode 100644
index 0000000..c7fa78f
--- /dev/null
+++ b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/status/reporters/TestStatusLogger.java
@@ -0,0 +1,209 @@
+/*
+ * 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.status.reporters;
+
+import org.apache.nifi.logging.LogLevel;
+import org.apache.nifi.minifi.bootstrap.QueryableStatusAggregator;
+import org.apache.nifi.minifi.commons.status.FlowStatusReport;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.slf4j.Logger;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
+import java.util.Properties;
+
+import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.ENCOUNTERED_IO_EXCEPTION;
+import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.LOGGING_LEVEL_KEY;
+import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.QUERY_KEY;
+import static org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger.REPORT_PERIOD_KEY;
+import static org.mockito.Mockito.verify;
+
+public class TestStatusLogger {
+
+    private static final String MOCK_STATUS = "FlowStatusReport{controllerServiceStatusList=null, processorStatusList=[{name='TailFile', processorHealth={runStatus='Running', hasBulletins=false, " +
+            "validationErrorList=[]}, processorStats=null, bulletinList=null}], connectionStatusList=null, remoteProcessingGroupStatusList=null, instanceStatus=null, systemDiagnosticsStatus=null," +
+            " reportingTaskStatusList=null, errorsGeneratingReport=[]}";
+
+    private static final String MOCK_QUERY = "processor:all:health";
+
+    private StatusLogger statusLogger;
+    private Logger logger;
+    private QueryableStatusAggregator queryableStatusAggregator;
+    private FlowStatusReport flowStatusReport;
+
+    @Before
+    public void init() throws IOException, NoSuchFieldException, IllegalAccessException {
+        statusLogger = Mockito.spy(new StatusLogger());
+
+        logger = Mockito.mock(Logger.class);
+        queryableStatusAggregator = Mockito.mock(QueryableStatusAggregator.class);
+        flowStatusReport = Mockito.mock(FlowStatusReport.class);
+
+        Mockito.when(flowStatusReport.toString()).thenReturn(MOCK_STATUS);
+
+        Field field = StatusLogger.class.getDeclaredField("logger");
+        field.setAccessible(true);
+        Field modifiersField = Field.class.getDeclaredField("modifiers");
+        modifiersField.setAccessible(true);
+        modifiersField.setInt(field, field.getModifiers() & ~Modifier.FINAL);
+        field.set(null, logger);
+
+
+        Mockito.when(queryableStatusAggregator.statusReport(MOCK_QUERY)).thenReturn(flowStatusReport);
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testFailedInitDueToFatalLogLevel(){
+        Properties properties = new Properties();
+        properties.setProperty(REPORT_PERIOD_KEY, "100");
+        properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.FATAL.name());
+        properties.setProperty(QUERY_KEY, MOCK_QUERY);
+
+        statusLogger.initialize(properties, queryableStatusAggregator);
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testFailedInitDueToNoPeriod(){
+        Properties properties = new Properties();
+        properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.INFO.name());
+        properties.setProperty(QUERY_KEY, MOCK_QUERY);
+
+        statusLogger.initialize(properties, queryableStatusAggregator);
+    }
+
+    @Test(expected = IllegalStateException.class)
+    public void testFailedInitDueToNoQuery(){
+        Properties properties = new Properties();
+        properties.setProperty(REPORT_PERIOD_KEY, "100");
+        properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.INFO.name());
+
+        statusLogger.initialize(properties, queryableStatusAggregator);
+    }
+
+    @Test
+    public void TestTrace() {
+        statusLogger.initialize(getProperties(LogLevel.TRACE), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).info(MOCK_STATUS, (Throwable) null);
+    }
+
+    @Test
+    public void TestDebug() {
+        statusLogger.initialize(getProperties(LogLevel.DEBUG), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).debug(MOCK_STATUS, (Throwable) null);
+    }
+
+    @Test
+    public void TestInfo() {
+        statusLogger.initialize(getProperties(LogLevel.INFO), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).info(MOCK_STATUS, (Throwable) null);
+    }
+
+    @Test
+    public void TestWarn() {
+        statusLogger.initialize(getProperties(LogLevel.WARN), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).warn(MOCK_STATUS, (Throwable) null);
+    }
+
+    @Test
+    public void TestError() {
+        statusLogger.initialize(getProperties(LogLevel.ERROR), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).error(MOCK_STATUS, (Throwable) null);
+    }
+
+    // Exception testing
+    @Test
+    public void TestTraceException() throws IOException {
+        Properties properties = new Properties();
+        properties.setProperty(REPORT_PERIOD_KEY, "100");
+        properties.setProperty(LOGGING_LEVEL_KEY, LogLevel.TRACE.name());
+        properties.setProperty(QUERY_KEY, MOCK_QUERY);
+
+        IOException ioException = new IOException("This is an expected test exception");
+        Mockito.when(queryableStatusAggregator.statusReport(MOCK_QUERY)).thenThrow(ioException);
+
+        statusLogger.initialize(properties, queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).trace(ENCOUNTERED_IO_EXCEPTION, ioException);
+    }
+
+    @Test
+    public void TestDebugException() throws IOException {
+        IOException ioException = new IOException("This is an expected test exception");
+        Mockito.when(queryableStatusAggregator.statusReport(MOCK_QUERY)).thenThrow(ioException);
+
+        statusLogger.initialize(getProperties(LogLevel.DEBUG), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).debug(ENCOUNTERED_IO_EXCEPTION, ioException);
+    }
+
+    @Test
+    public void TestInfoException() throws IOException {
+        IOException ioException = new IOException("This is an expected test exception");
+        Mockito.when(queryableStatusAggregator.statusReport(MOCK_QUERY)).thenThrow(ioException);
+
+        statusLogger.initialize(getProperties(LogLevel.INFO), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).info(ENCOUNTERED_IO_EXCEPTION, ioException);
+    }
+
+    @Test
+    public void TestWarnException() throws IOException {
+        IOException ioException = new IOException("This is an expected test exception");
+        Mockito.when(queryableStatusAggregator.statusReport(MOCK_QUERY)).thenThrow(ioException);
+
+        statusLogger.initialize(getProperties(LogLevel.WARN), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).warn(ENCOUNTERED_IO_EXCEPTION, ioException);
+    }
+
+    @Test
+    public void TestErrorException() throws IOException {
+        IOException ioException = new IOException("This is an expected test exception");
+        Mockito.when(queryableStatusAggregator.statusReport(MOCK_QUERY)).thenThrow(ioException);
+
+        statusLogger.initialize(getProperties(LogLevel.ERROR), queryableStatusAggregator);
+        statusLogger.start();
+
+        verify(logger, Mockito.timeout(300).atLeastOnce()).error(ENCOUNTERED_IO_EXCEPTION, ioException);
+    }
+
+    private static Properties getProperties(LogLevel logLevel){
+        Properties properties = new Properties();
+        properties.setProperty(REPORT_PERIOD_KEY, "100");
+        properties.setProperty(LOGGING_LEVEL_KEY, logLevel.name());
+        properties.setProperty(QUERY_KEY, MOCK_QUERY);
+        return properties;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-docs/src/main/markdown/System_Admin_Guide.md
----------------------------------------------------------------------
diff --git a/minifi-docs/src/main/markdown/System_Admin_Guide.md b/minifi-docs/src/main/markdown/System_Admin_Guide.md
index a27bde3..a6c13f9 100644
--- a/minifi-docs/src/main/markdown/System_Admin_Guide.md
+++ b/minifi-docs/src/main/markdown/System_Admin_Guide.md
@@ -21,13 +21,81 @@
 
 [NiFi Homepage](http://nifi.apache.org)
 
-# FlowStatus Query Options
+# Status Reporting and Querying
 
-From the minifi.sh script there is the ability to query to get the current status of the flow. This section will give an overview of the different options.
+Since MiNiFi does not have a UI there needed to be ways to get the information normally seen in the NiFi UI, information such as the stats or bulletins of a processor. This FlowStatus can be queried using using the MiNiFi.sh script or by configuring one of the Periodic Status Reporters. The API for the query is the same for the reporters and the flowStatus script option. The API is outlined in the "FlowStatus Query Options" section.
 
-Note: Currently the script only accepts one high level option at a time. Also any names of connections, remote process groups or processors that contain " " (a space), ":", ";" or "," will cause parsing errors when querying.
+## FlowStatus Script Query
 
-## Processors
+From the minifi.sh script there is the ability to manually query to get the current status of the flow. Currently the script only accepts one high level option at a time. Also any names of connections, remote process groups or processors that contain " " (a space), ":", ";" or "," will cause parsing errors when querying. Below is an example query.
+
+```
+minifi.sh flowStatus processor:TailFile:health,stats,bulletins
+```
+
+## Periodic Status Reporters
+
+In addition to manually querying via the MiNiFi.sh script, Periodic Status Reporters can be set up to periodically report the status of the flow. Every period the configured query will be executed and the result reported via the configured implementation. The Reporters are configured in the bootstrap.conf using the "nifi.minifi.status.reporter.components" key followed by the full path name of the desired Reporter implementation to run (a comma separated list can be used to define more than one). For example:
+
+```
+nifi.minifi.status.reporter.components=org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger
+```
+
+
+### StatusLogger
+
+class name: org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger
+
+This Periodic Status Reporter simply logs the results of the query to the logs. By default it will be logged to the minifi-bootstrap.log but using the logback.xml it can be configured to log to wherever is desired.
+
+Option | Description
+------ | -----------
+nifi.minifi.status.reporter.log.query | The FlowStatus query to run.
+nifi.minifi.status.reporter.log.level | The log level at which to log the status. Available options are "TRACE", "DEBUG", "INFO", "WARN" and "ERROR".
+nifi.minifi.status.reporter.log.period | The delay (in milliseconds) between each query.
+
+Example bootstrap.conf configuration:
+
+```
+# The FlowStatus query to submit to the MiNiFi instance
+nifi.minifi.status.reporter.log.query=instance:health,bulletins
+# The log level at which the status will be logged
+nifi.minifi.status.reporter.log.level=INFO
+# The period (in milliseconds) at which to log the status
+nifi.minifi.status.reporter.log.period=60000
+```
+
+Example logback.xml configuration to output the status to it's own rolling log file:
+
+```
+<appender name="STATUS_LOG_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
+    <file>logs/minifi-status.log</file>
+    <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
+        <!--
+          For daily rollover, use 'user_%d.log'.
+          For hourly rollover, use 'user_%d{yyyy-MM-dd_HH}.log'.
+          To GZIP rolled files, replace '.log' with '.log.gz'.
+          To ZIP rolled files, replace '.log' with '.log.zip'.
+        -->
+        <fileNamePattern>./logs/minifi-bootstrap_%d.log</fileNamePattern>
+        <!-- keep 5 log files worth of history -->
+        <maxHistory>5</maxHistory>
+    </rollingPolicy>
+    <encoder class="ch.qos.logback.classic.encoder.PatternLayoutEncoder">
+        <pattern>%date %level [%thread] %logger{40} %msg%n</pattern>
+    </encoder>
+</appender>
+
+<logger name="org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger" level="INFO" additivity="false">
+    <appender-ref ref="STATUS_LOG_FILE" />
+</logger>
+```
+
+## FlowStatus Query Options
+
+This section gives outlines each of the different options to query the MiNiFi instance for the FlowStatus.
+
+### Processors
 
 To query the processors use the "processor" flag followed by the id of the processor to get (or "all") followed by one of the processor options. The processor options are below.
 
@@ -41,7 +109,7 @@ An example query to get the health, bulletins and stats of the "TailFile" proces
 ```
 minifi.sh flowStatus processor:TailFile:health,stats,bulletins
 ```
-## Connections
+### Connections
 
 To query the connections use the "connection" flag followed by the id of the connection to get (or "all") followed by one of the connection options. The connection options are below.
 
@@ -55,7 +123,7 @@ An example query to get the health and stats of the "TailToS2S" connection is be
 minifi.sh flowStatus connection:TailToS2S:health,stats
 ```
 
-## Remote Process Groups
+### Remote Process Groups
 
 To query the remote process groups (RPG) use the "remoteProcessGroup" flag followed by the id of the remote process group to get (or "all") followed by one of the remote process group options. The remote process group options are below.
 
@@ -73,7 +141,7 @@ An example query to get the health, bulletins, authorization issues, input ports
 minifi.sh flowStatus remoteprocessinggroup:all:health,bulletins,authorizationIssues,inputports,stats
 ```
 
-## Controller Services
+### Controller Services
 
 To query the controller services use the "controllerServices" flag followed by one of the controller service options. The controller service options are below.
 
@@ -88,7 +156,7 @@ An example query to get the health and bulletins of all the controller services
 minifi.sh flowStatus controllerservices:health,bulletins
 ```
 
-## Provenance Reporting
+### Provenance Reporting
 
 To query the status of the provenance reporting use the "provenancereporting" flag followed by one of the provenance reporting  options. The provenance reporting options are below.
 
@@ -103,7 +171,7 @@ An example query to get the health and bulletins of the provenance reporting is
 minifi.sh flowStatus provenancereporting:health,bulletins
 ```
 
-## Instance
+### Instance
 
 To query the status of the MiNiFi instance in general use the "instance" flag followed by one of the instance options. The instance options are below.
 
@@ -119,7 +187,7 @@ An example query to get the health, stats and bulletins of the instance is below
 minifi.sh flowStatus instance:health,stats,bulletins
 ```
 
-## System Diagnostics
+### System Diagnostics
 
 To query the system diagnostics use the "systemdiagnostics" flag followed by one of the system diagnostics options. The system diagnostics options are below.
 
@@ -137,7 +205,7 @@ An example query to get the heap, processor stats, content repository usage, Flo
 minifi.sh flowStatus systemdiagnostics:heap,processorstats,contentrepositoryusage,flowfilerepositoryusage,garbagecollection
 ```
 
-## Example
+### Example
 
 This is an example of a simple query to get the health of all the processors and its results from a simple flow:
 
@@ -149,13 +217,12 @@ MiNiFi home: /Users/user/projects/nifi-minifi/minifi-assembly/target/minifi-0.0.
 
 Bootstrap Config File: /Users/user/projects/nifi-minifi/minifi-assembly/target/minifi-0.0.1-SNAPSHOT-bin/minifi-0.0.1-SNAPSHOT/conf/bootstrap.conf
 
-Args
-flowStatus
-processor:all:health
 FlowStatusReport{controllerServiceStatusList=null, processorStatusList=[{name='TailFile', processorHealth={runStatus='Running', hasBulletins=false, validationErrorList=[]}, processorStats=null,
 bulletinList=null}], connectionStatusList=null, remoteProcessingGroupStatusList=null, instanceStatus=null, systemDiagnosticsStatus=null, reportingTaskStatusList=null, errorsGeneratingReport=[]}
 ```
 
+# Periodic Status Reporters
+
 
 # Config File
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/291b171c/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
index a365c90..7691859 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/bootstrap.conf
@@ -47,6 +47,18 @@ nifi.minifi.config=./conf/config.yml
 # Port on which the Jetty server will bind to, keep commented for a random open port
 #nifi.minifi.notifier.http.port=8338
 
+# Periodic Status Reporters to use for the associated agent, comma separated list of class names
+#nifi.minifi.status.reporter.components=org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger
+
+# Periodic Status Logger configuration
+
+# The FlowStatus query to submit to the MiNiFi instance
+#nifi.minifi.status.reporter.log.query=instance:health,bulletins
+# The log level at which the status will be logged
+#nifi.minifi.status.reporter.log.level=INFO
+# The period (in milliseconds) at which to log the status
+#nifi.minifi.status.reporter.log.period=60000
+
 # Disable JSR 199 so that we can use JSP's without running a JDK
 java.arg.1=-Dorg.apache.jasper.compiler.disablejsr199=true
 


[2/3] nifi-minifi git commit: MINIFI-39: Minor Admin Guide edits.

Posted by jp...@apache.org.
MINIFI-39: Minor Admin Guide edits.


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

Branch: refs/heads/master
Commit: 8d8c26387fbdfbaa8a61bdc2cc20564d34f8656e
Parents: 291b171
Author: Sarah Olson <so...@hortonworks.com>
Authored: Wed Oct 19 17:36:44 2016 -0400
Committer: Joseph Percivall <jo...@yahoo.com>
Committed: Thu Oct 20 15:25:57 2016 -0400

----------------------------------------------------------------------
 .../src/main/markdown/System_Admin_Guide.md     | 21 ++++++++++----------
 1 file changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/8d8c2638/minifi-docs/src/main/markdown/System_Admin_Guide.md
----------------------------------------------------------------------
diff --git a/minifi-docs/src/main/markdown/System_Admin_Guide.md b/minifi-docs/src/main/markdown/System_Admin_Guide.md
index a6c13f9..e5c97b9 100644
--- a/minifi-docs/src/main/markdown/System_Admin_Guide.md
+++ b/minifi-docs/src/main/markdown/System_Admin_Guide.md
@@ -23,19 +23,20 @@
 
 # Status Reporting and Querying
 
-Since MiNiFi does not have a UI there needed to be ways to get the information normally seen in the NiFi UI, information such as the stats or bulletins of a processor. This FlowStatus can be queried using using the MiNiFi.sh script or by configuring one of the Periodic Status Reporters. The API for the query is the same for the reporters and the flowStatus script option. The API is outlined in the "FlowStatus Query Options" section.
+You can use FlowStatus to view status, processor bulletin, and health information similar to that displayed in the NiFi UI. Query FlowStatus either using the MiNiFi.sh script or by configuring one of the Periodic Status Reporters. The API for the query is the same for the reporters and the flowStatus script option. The API is outlined in the "FlowStatus Query Options" section below.
 
 ## FlowStatus Script Query
 
-From the minifi.sh script there is the ability to manually query to get the current status of the flow. Currently the script only accepts one high level option at a time. Also any names of connections, remote process groups or processors that contain " " (a space), ":", ";" or "," will cause parsing errors when querying. Below is an example query.
+From the minifi.sh script, you can manually query to get the current status of your  dataflow. The following is an example of a minifi.sh query you might run to view health, stats, and bulletins for the TailFile processor. This query returns information to your command-line.
 
 ```
 minifi.sh flowStatus processor:TailFile:health,stats,bulletins
 ```
+Currently the script only accepts one high level option at a time. Also any names of connections, remote process groups, or processors that contain " " (a space), ":", ";" or "," cause parsing errors when querying.
 
 ## Periodic Status Reporters
 
-In addition to manually querying via the MiNiFi.sh script, Periodic Status Reporters can be set up to periodically report the status of the flow. Every period the configured query will be executed and the result reported via the configured implementation. The Reporters are configured in the bootstrap.conf using the "nifi.minifi.status.reporter.components" key followed by the full path name of the desired Reporter implementation to run (a comma separated list can be used to define more than one). For example:
+You can set up Periodic Status Reporters to periodically report the status of your dataflow. The query executes at configurable intervals and the results are reported using the configured implementation. Configure the Reporters in the bootstrap.conf file, using the "nifi.minifi.status.reporter.components" key followed by the full path name of the desired Reporter implementation to run. Use a comma separated list  to define more than one Reporter implementation. For example:
 
 ```
 nifi.minifi.status.reporter.components=org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger
@@ -46,7 +47,7 @@ nifi.minifi.status.reporter.components=org.apache.nifi.minifi.bootstrap.status.r
 
 class name: org.apache.nifi.minifi.bootstrap.status.reporters.StatusLogger
 
-This Periodic Status Reporter simply logs the results of the query to the logs. By default it will be logged to the minifi-bootstrap.log but using the logback.xml it can be configured to log to wherever is desired.
+The Periodic Status Reporter logs the results of the query to the logs. By default it is logged to the minifi-bootstrap.log but you can modify logback.xml to log to an alternate file and location.
 
 Option | Description
 ------ | -----------
@@ -65,7 +66,7 @@ nifi.minifi.status.reporter.log.level=INFO
 nifi.minifi.status.reporter.log.period=60000
 ```
 
-Example logback.xml configuration to output the status to it's own rolling log file:
+Example logback.xml configuration to output the status to its own rolling log file:
 
 ```
 <appender name="STATUS_LOG_FILE" class="ch.qos.logback.core.rolling.RollingFileAppender">
@@ -93,11 +94,11 @@ Example logback.xml configuration to output the status to it's own rolling log f
 
 ## FlowStatus Query Options
 
-This section gives outlines each of the different options to query the MiNiFi instance for the FlowStatus.
+This section outlines each option to query the MiNiFi instance for the FlowStatus.
 
 ### Processors
 
-To query the processors use the "processor" flag followed by the id of the processor to get (or "all") followed by one of the processor options. The processor options are below.
+To query the processors use the "processor" flag followed by the processor ID to get (or "all") followed by one of the processor options. The processor options are below.
 
 Option | Description
 ------ | -----------
@@ -111,7 +112,7 @@ minifi.sh flowStatus processor:TailFile:health,stats,bulletins
 ```
 ### Connections
 
-To query the connections use the "connection" flag followed by the id of the connection to get (or "all") followed by one of the connection options. The connection options are below.
+To query the connections use the "connection" flag followed by the connection ID to get (or "all") followed by one of the connection options. The connection options are below.
 
 Option | Description
 ------ | -----------
@@ -125,7 +126,7 @@ minifi.sh flowStatus connection:TailToS2S:health,stats
 
 ### Remote Process Groups
 
-To query the remote process groups (RPG) use the "remoteProcessGroup" flag followed by the id of the remote process group to get (or "all") followed by one of the remote process group options. The remote process group options are below.
+To query the remote process groups (RPG) use the "remoteProcessGroup" flag followed by the RPG ID to get (or "all") followed by one of the remote process group options. The remote process group options are below.
 
 Option | Description
 ------ | -----------
@@ -173,7 +174,7 @@ minifi.sh flowStatus provenancereporting:health,bulletins
 
 ### Instance
 
-To query the status of the MiNiFi instance in general use the "instance" flag followed by one of the instance options. The instance options are below.
+To query the status of the MiNiFi instance use the "instance" flag followed by one of the instance options. The instance options are below.
 
 Option | Description
 ------ | -----------


[3/3] nifi-minifi git commit: MINIFI-39 Responding to review comments

Posted by jp...@apache.org.
MINIFI-39 Responding to review comments

This closes #46


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

Branch: refs/heads/master
Commit: 4e1363a2eb6f8c9e54db4c5844ce68fb2105c35a
Parents: 8d8c263
Author: Joseph Percivall <jo...@yahoo.com>
Authored: Thu Oct 20 14:26:27 2016 -0400
Committer: Joseph Percivall <jo...@yahoo.com>
Committed: Thu Oct 20 15:26:00 2016 -0400

----------------------------------------------------------------------
 .../nifi/minifi/bootstrap/status/PeriodicStatusReporter.java     | 2 +-
 minifi-docs/src/main/markdown/System_Admin_Guide.md              | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/4e1363a2/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java
index 8f0059c..916c903 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/status/PeriodicStatusReporter.java
@@ -46,7 +46,7 @@ public abstract class PeriodicStatusReporter {
      */
     public void start() {
         if (reportRunner == null){
-            throw new IllegalStateException("Programmatic error, the reportRunner has is still NULL when 'start' was called.");
+            throw new IllegalStateException("Programmatic error, the reportRunner is still NULL when 'start' was called.");
         }
         scheduledExecutorService.scheduleAtFixedRate(reportRunner, period, period, TimeUnit.MILLISECONDS);
     }

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/4e1363a2/minifi-docs/src/main/markdown/System_Admin_Guide.md
----------------------------------------------------------------------
diff --git a/minifi-docs/src/main/markdown/System_Admin_Guide.md b/minifi-docs/src/main/markdown/System_Admin_Guide.md
index e5c97b9..14ce249 100644
--- a/minifi-docs/src/main/markdown/System_Admin_Guide.md
+++ b/minifi-docs/src/main/markdown/System_Admin_Guide.md
@@ -19,11 +19,11 @@
 
 [Apache NiFi Team](dev@nifi.apache.org>)
 
-[NiFi Homepage](http://nifi.apache.org)
+[MiNiFi Homepage](https://nifi.apache.org/minifi/index.html)
 
 # Status Reporting and Querying
 
-You can use FlowStatus to view status, processor bulletin, and health information similar to that displayed in the NiFi UI. Query FlowStatus either using the MiNiFi.sh script or by configuring one of the Periodic Status Reporters. The API for the query is the same for the reporters and the flowStatus script option. The API is outlined in the "FlowStatus Query Options" section below.
+In NiFi there is a lot of information, such as stats and bulletins, that is only available to view through the UI. MiNiFi provides access to this information through a query mechanism. You can query FlowStatus either using the MiNiFi.sh script or by configuring one of the Periodic Status Reporters. The API for the query is the same for the reporters and the "flowStatus" script option. The API is outlined in the "FlowStatus Query Options" section below.
 
 ## FlowStatus Script Query