You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by mc...@apache.org on 2015/04/28 16:05:02 UTC

[28/50] [abbrv] incubator-nifi git commit: NIFI-271

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java
index 9e429d6..fe34fe0 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/VolatileFlowFileRepository.java
@@ -26,9 +26,7 @@ import org.apache.nifi.controller.repository.claim.ContentClaimManager;
 
 /**
  * <p>
- * An in-memory implementation of the {@link FlowFileRepository}. Upon restart,
- * all FlowFiles will be discarded, including those that have been swapped out
- * by a {@link FlowFileSwapManager}.
+ * An in-memory implementation of the {@link FlowFileRepository}. Upon restart, all FlowFiles will be discarded, including those that have been swapped out by a {@link FlowFileSwapManager}.
  * </p>
  */
 public class VolatileFlowFileRepository implements FlowFileRepository {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
index 0779c4d..f2df821 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java
@@ -62,21 +62,12 @@ import org.wali.WriteAheadRepository;
  * </p>
  *
  * <p>
- * We expose a property named <code>nifi.flowfile.repository.always.sync</code>
- * that is a boolean value indicating whether or not to force WALI to sync with
- * disk on each update. By default, the value is <code>false</code>. This is
- * needed only in situations in which power loss is expected and not mitigated
- * by Uninterruptable Power Sources (UPS) or when running in an unstable Virtual
- * Machine for instance. Otherwise, we will flush the data that is written to
- * the Operating System and the Operating System will be responsible to flush
- * its buffers when appropriate. The Operating System can be configured to hold
- * only a certain buffer size or not to buffer at all, as well. When using a
- * UPS, this is generally not an issue, as the machine is typically notified
- * before dying, in which case the Operating System will flush the data to disk.
- * Additionally, most disks on enterprise servers also have battery backups that
- * can power the disks long enough to flush their buffers. For this reason, we
- * choose instead to not sync to disk for every write but instead sync only when
- * we checkpoint.
+ * We expose a property named <code>nifi.flowfile.repository.always.sync</code> that is a boolean value indicating whether or not to force WALI to sync with disk on each update. By default, the value
+ * is <code>false</code>. This is needed only in situations in which power loss is expected and not mitigated by Uninterruptable Power Sources (UPS) or when running in an unstable Virtual Machine for
+ * instance. Otherwise, we will flush the data that is written to the Operating System and the Operating System will be responsible to flush its buffers when appropriate. The Operating System can be
+ * configured to hold only a certain buffer size or not to buffer at all, as well. When using a UPS, this is generally not an issue, as the machine is typically notified before dying, in which case
+ * the Operating System will flush the data to disk. Additionally, most disks on enterprise servers also have battery backups that can power the disks long enough to flush their buffers. For this
+ * reason, we choose instead to not sync to disk for every write but instead sync only when we checkpoint.
  * </p>
  */
 public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncListener {
@@ -263,9 +254,7 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis
     }
 
     /**
-     * Swaps the FlowFiles that live on the given Connection out to disk, using
-     * the specified Swap File and returns the number of FlowFiles that were
-     * persisted.
+     * Swaps the FlowFiles that live on the given Connection out to disk, using the specified Swap File and returns the number of FlowFiles that were persisted.
      *
      * @param queue queue to swap out
      * @param swapLocation location to swap to

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentDirection.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentDirection.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentDirection.java
index b5d70b2..83901ef 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentDirection.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/ContentDirection.java
@@ -17,19 +17,16 @@
 package org.apache.nifi.controller.repository.claim;
 
 /**
- * Specifies one side of the Provenance Event for which the Content Claim is
- * being referenced
+ * Specifies one side of the Provenance Event for which the Content Claim is being referenced
  */
 public enum ContentDirection {
 
     /**
-     * Indicates the Content Claim that was the Input to the Process that
-     * generating a Provenance Event
+     * Indicates the Content Claim that was the Input to the Process that generating a Provenance Event
      */
     INPUT,
     /**
-     * Indicates the Content Claim that is the Output of the process that
-     * generated the Provenance Event.
+     * Indicates the Content Claim that is the Output of the process that generated the Provenance Event.
      */
     OUTPUT;
 }

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
index 54a1b2c..a8a6963 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
@@ -20,9 +20,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * <p>
- * A ContentClaim is a reference to a given flow file's content. Multiple flow
- * files may reference the same content by both having the same content
- * claim.</p>
+ * A ContentClaim is a reference to a given flow file's content. Multiple flow files may reference the same content by both having the same content claim.</p>
  *
  * <p>
  * Must be thread safe</p>
@@ -88,8 +86,7 @@ public final class StandardContentClaim implements ContentClaim, Comparable<Cont
     }
 
     /**
-     * Provides the natural ordering for ContentClaim objects. By default they
-     * are sorted by their id, then container, then section
+     * Provides the natural ordering for ContentClaim objects. By default they are sorted by their id, then container, then section
      *
      * @param other other claim
      * @return x such that x <=1 if this is less than other;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java
index ddcf6c9..1ab85ff 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseInputStream.java
@@ -20,9 +20,7 @@ import java.io.IOException;
 import java.io.InputStream;
 
 /**
- * Wraps an existing InputStream, so that when {@link InputStream#close()} is
- * called, the underlying InputStream is NOT closed but this InputStream can no
- * longer be written to
+ * Wraps an existing InputStream, so that when {@link InputStream#close()} is called, the underlying InputStream is NOT closed but this InputStream can no longer be written to
  */
 public class DisableOnCloseInputStream extends InputStream {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseOutputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseOutputStream.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseOutputStream.java
index 720e7f5..4845d60 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseOutputStream.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/DisableOnCloseOutputStream.java
@@ -20,9 +20,7 @@ import java.io.IOException;
 import java.io.OutputStream;
 
 /**
- * Wraps an existing OutputStream, so that when {@link OutputStream#close()} is
- * called, the underlying OutputStream is NOT closed but this OutputStream can
- * no longer be written to
+ * Wraps an existing OutputStream, so that when {@link OutputStream#close()} is called, the underlying OutputStream is NOT closed but this OutputStream can no longer be written to
  */
 public class DisableOnCloseOutputStream extends OutputStream {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessInputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessInputStream.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessInputStream.java
index a710070..946f042 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessInputStream.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessInputStream.java
@@ -27,12 +27,8 @@ import org.apache.nifi.processor.exception.FlowFileAccessException;
 
 /**
  * <p>
- * Wraps an InputStream so that if any IOException is thrown, it will be wrapped
- * in a FlowFileAccessException. We do this to isolate IOExceptions thrown by
- * the framework from those thrown by user code. If thrown by the framework, it
- * generally indicates a problem communicating with the Content Repository and
- * session rollback is often appropriate so that the FlowFile can be processed
- * again.
+ * Wraps an InputStream so that if any IOException is thrown, it will be wrapped in a FlowFileAccessException. We do this to isolate IOExceptions thrown by the framework from those thrown by user
+ * code. If thrown by the framework, it generally indicates a problem communicating with the Content Repository and session rollback is often appropriate so that the FlowFile can be processed again.
  * </p>
  */
 public class FlowFileAccessInputStream extends FilterInputStream {
@@ -56,8 +52,7 @@ public class FlowFileAccessInputStream extends FilterInputStream {
     }
 
     /**
-     * @return the ContentNotFoundException that was thrown by this stream, or
-     * <code>null</code> if no such Exception was thrown
+     * @return the ContentNotFoundException that was thrown by this stream, or <code>null</code> if no such Exception was thrown
      */
     public ContentNotFoundException getContentNotFoundException() {
         return thrown;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessOutputStream.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessOutputStream.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessOutputStream.java
index 744e3a6..f4edcfe 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessOutputStream.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/FlowFileAccessOutputStream.java
@@ -25,12 +25,9 @@ import org.apache.nifi.processor.exception.FlowFileAccessException;
 
 /**
  * <p>
- * Wraps an OutputStream so that if any IOException is thrown, it will be
- * wrapped in a FlowFileAccessException. We do this to isolate IOExceptions
- * thrown by the framework from those thrown by user code. If thrown by the
- * framework, it generally indicates a problem communicating with the Content
- * Repository (such as out of disk space) and session rollback is often
- * appropriate so that the FlowFile can be processed again.
+ * Wraps an OutputStream so that if any IOException is thrown, it will be wrapped in a FlowFileAccessException. We do this to isolate IOExceptions thrown by the framework from those thrown by user
+ * code. If thrown by the framework, it generally indicates a problem communicating with the Content Repository (such as out of disk space) and session rollback is often appropriate so that the
+ * FlowFile can be processed again.
  * </p>
  */
 public class FlowFileAccessOutputStream extends FilterOutputStream {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java
index 932cf9c..bd5fec1 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/io/LongHolder.java
@@ -17,10 +17,8 @@
 package org.apache.nifi.controller.repository.io;
 
 /**
- * Class to hold a long value that can be incremented and decremented. This
- * allows the abstraction of passing a long value by reference, rather than by
- * value, without the overhead of synchronization required by the use of an
- * AtomicLong.
+ * Class to hold a long value that can be incremented and decremented. This allows the abstraction of passing a long value by reference, rather than by value, without the overhead of synchronization
+ * required by the use of an AtomicLong.
  */
 public class LongHolder {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
index 01285b0..7617e7c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java
@@ -39,8 +39,7 @@ import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.processor.exception.ProcessException;
 
 /**
- * This class is essentially an empty shell for {@link Connectable}s that are
- * not Processors
+ * This class is essentially an empty shell for {@link Connectable}s that are not Processors
  */
 public class ConnectableProcessContext implements ProcessContext {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
index cb7f55f..ea0b456 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ScheduleState.java
@@ -62,12 +62,11 @@ public class ScheduleState {
     }
 
     /**
-     * Maintains an AtomicBoolean so that the first thread to call this method
-     * after a Processor is no longer scheduled to run will receive a
-     * <code>true</code> and MUST call the methods annotated with @OnStopped
+     * Maintains an AtomicBoolean so that the first thread to call this method after a Processor is no longer scheduled to run will receive a <code>true</code> and MUST call the methods annotated with
      *
-     * @return <code>true</code> if the caller is required to call Processor
-     * methods annotated with
+     * @OnStopped
+     *
+     * @return <code>true</code> if the caller is required to call Processor methods annotated with
      * @OnStopped, <code>false</code> otherwise
      */
     public boolean mustCallOnStoppedMethods() {
@@ -75,8 +74,7 @@ public class ScheduleState {
     }
 
     /**
-     * Establishes the list of relevant futures for this processor. Replaces any
-     * previously held futures.
+     * Establishes the list of relevant futures for this processor. Replaces any previously held futures.
      *
      * @param newFutures futures
      */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
index bb565cb..ffa669d 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/StandardProcessScheduler.java
@@ -69,8 +69,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Responsible for scheduling Processors, Ports, and Funnels to run at regular
- * intervals
+ * Responsible for scheduling Processors, Ports, and Funnels to run at regular intervals
  */
 public final class StandardProcessScheduler implements ProcessScheduler {
 
@@ -257,10 +256,8 @@ public final class StandardProcessScheduler implements ProcessScheduler {
     }
 
     /**
-     * Starts scheduling the given processor to run after invoking all methods
-     * on the underlying {@link nifi.processor.Processor
-     * FlowFileProcessor} that are annotated with the {@link OnScheduled}
-     * annotation.
+     * Starts scheduling the given processor to run after invoking all methods on the underlying {@link nifi.processor.Processor
+     * FlowFileProcessor} that are annotated with the {@link OnScheduled} annotation.
      */
     @Override
     public synchronized void startProcessor(final ProcessorNode procNode) {
@@ -379,9 +376,8 @@ public final class StandardProcessScheduler implements ProcessScheduler {
     }
 
     /**
-     * Stops scheduling the given processor to run and invokes all methods on
-     * the underlying {@link nifi.processor.Processor FlowFileProcessor} that
-     * are annotated with the {@link OnUnscheduled} annotation.
+     * Stops scheduling the given processor to run and invokes all methods on the underlying {@link nifi.processor.Processor FlowFileProcessor} that are annotated with the {@link OnUnscheduled}
+     * annotation.
      */
     @Override
     public synchronized void stopProcessor(final ProcessorNode procNode) {
@@ -577,9 +573,7 @@ public final class StandardProcessScheduler implements ProcessScheduler {
     }
 
     /**
-     * Returns the ScheduleState that is registered for the given component; if
-     * no ScheduleState current is registered, one is created and registered
-     * atomically, and then that value is returned.
+     * Returns the ScheduleState that is registered for the given component; if no ScheduleState current is registered, one is created and registered atomically, and then that value is returned.
      *
      * @param schedulable schedulable
      * @return scheduled state

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
index a45bf76..d8506c1 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/service/StandardControllerServiceProvider.java
@@ -518,10 +518,8 @@ public class StandardControllerServiceProvider implements ControllerServiceProvi
     }
 
     /**
-     * Returns a List of all components that reference the given referencedNode
-     * (either directly or indirectly through another service) that are also of
-     * the given componentType. The list that is returned is in the order in
-     * which they will need to be 'activated' (enabled/started).
+     * Returns a List of all components that reference the given referencedNode (either directly or indirectly through another service) that are also of the given componentType. The list that is
+     * returned is in the order in which they will need to be 'activated' (enabled/started).
      *
      * @param referencedNode node
      * @param componentType type

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardStatusSnapshot.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardStatusSnapshot.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardStatusSnapshot.java
index e1fdca8..abaf899 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardStatusSnapshot.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/StandardStatusSnapshot.java
@@ -53,7 +53,7 @@ public class StandardStatusSnapshot implements StatusSnapshot {
             public StatusSnapshot reduce(final List<StatusSnapshot> values) {
                 Date reducedTimestamp = null;
                 final Set<MetricDescriptor<?>> allDescriptors = new LinkedHashSet<>(metricValues.keySet());
-                
+
                 for (final StatusSnapshot statusSnapshot : values) {
                     if (reducedTimestamp == null) {
                         reducedTimestamp = statusSnapshot.getTimestamp();

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
index f3cbb90..a824ad0 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunConnectableTask.java
@@ -35,9 +35,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Continually runs a Connectable as long as the processor has work to do.
- * {@link #call()} will return <code>true</code> if the Connectable should be
- * yielded, <code>false</code> otherwise.
+ * Continually runs a Connectable as long as the processor has work to do. {@link #call()} will return <code>true</code> if the Connectable should be yielded, <code>false</code> otherwise.
  */
 public class ContinuallyRunConnectableTask implements Callable<Boolean> {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
index baed6ae..efa5814 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ContinuallyRunProcessorTask.java
@@ -44,9 +44,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Continually runs a processor as long as the processor has work to do.
- * {@link #call()} will return <code>true</code> if the processor should be
- * yielded, <code>false</code> otherwise.
+ * Continually runs a processor as long as the processor has work to do. {@link #call()} will return <code>true</code> if the processor should be yielded, <code>false</code> otherwise.
  */
 public class ContinuallyRunProcessorTask implements Callable<Boolean> {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
index a351a68..7d8bcec 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/tasks/ExpireFlowFiles.java
@@ -34,9 +34,7 @@ import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.util.FormatUtils;
 
 /**
- * This task runs through all Connectable Components and goes through its
- * incoming queues, polling for FlowFiles and accepting none. This causes the
- * desired side effect of expiring old FlowFiles.
+ * This task runs through all Connectable Components and goes through its incoming queues, polling for FlowFiles and accepting none. This causes the desired side effect of expiring old FlowFiles.
  */
 public class ExpireFlowFiles implements Runnable {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
index fccd10e..5de1beb 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/encrypt/StringEncryptor.java
@@ -26,16 +26,14 @@ import org.jasypt.exceptions.EncryptionOperationNotPossibleException;
 
 /**
  * <p>
- * An application specific string encryptor that collects configuration from the
- * application properties, system properties, and/or system environment.
+ * An application specific string encryptor that collects configuration from the application properties, system properties, and/or system environment.
  * </p>
  *
  * <p>
  * Instance of this class are thread-safe</p>
  *
  * <p>
- * The encryption provider and algorithm is configured using the application
- * properties:
+ * The encryption provider and algorithm is configured using the application properties:
  * <ul>
  * <li>nifi.sensitive.props.provider</li>
  * <li>nifi.sensitive.props.algorithm</li>
@@ -73,12 +71,10 @@ public final class StringEncryptor {
     }
 
     /**
-     * Creates an instance of the nifi sensitive property encryptor. Validates
-     * that the encryptor is actually working.
+     * Creates an instance of the nifi sensitive property encryptor. Validates that the encryptor is actually working.
      *
      * @return encryptor
-     * @throws EncryptionException if any issues arise initializing or
-     * validating the encryptor
+     * @throws EncryptionException if any issues arise initializing or validating the encryptor
      */
     public static StringEncryptor createEncryptor() throws EncryptionException {
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/engine/FlowEngine.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/engine/FlowEngine.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/engine/FlowEngine.java
index 3be178f..d407e2f 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/engine/FlowEngine.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/engine/FlowEngine.java
@@ -34,8 +34,7 @@ public final class FlowEngine extends ScheduledThreadPoolExecutor {
     /**
      * Creates a new instance of FlowEngine
      *
-     * @param corePoolSize the maximum number of threads available to tasks
-     * running in the engine.
+     * @param corePoolSize the maximum number of threads available to tasks running in the engine.
      * @param threadNamePrefix for naming the thread
      */
     public FlowEngine(int corePoolSize, final String threadNamePrefix) {
@@ -45,11 +44,9 @@ public final class FlowEngine extends ScheduledThreadPoolExecutor {
     /**
      * Creates a new instance of FlowEngine
      *
-     * @param corePoolSize the maximum number of threads available to tasks
-     * running in the engine.
+     * @param corePoolSize the maximum number of threads available to tasks running in the engine.
      * @param threadNamePrefix for thread naming
-     * @param daemon if true, the thread pool will be populated with daemon
-     * threads, otherwise the threads will not be marked as daemon.
+     * @param daemon if true, the thread pool will be populated with daemon threads, otherwise the threads will not be marked as daemon.
      */
     public FlowEngine(int corePoolSize, final String threadNamePrefix, final boolean daemon) {
         super(corePoolSize);
@@ -70,8 +67,7 @@ public final class FlowEngine extends ScheduledThreadPoolExecutor {
     }
 
     /**
-     * Hook method called by the running thread whenever a runnable task is
-     * given to the thread to run.
+     * Hook method called by the running thread whenever a runnable task is given to the thread to run.
      *
      * @param thread thread
      * @param runnable runnable
@@ -84,9 +80,7 @@ public final class FlowEngine extends ScheduledThreadPoolExecutor {
     }
 
     /**
-     * Hook method called by the thread that executed the given runnable after
-     * execution of the runnable completed. Logs the fact of completion and any
-     * errors that might have occured.
+     * Hook method called by the thread that executed the given runnable after execution of the runnable completed. Logs the fact of completion and any errors that might have occured.
      *
      * @param runnable runnable
      * @param throwable throwable

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
index e8708bd..a20e974 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/events/VolatileBulletinRepository.java
@@ -176,9 +176,8 @@ public class VolatileBulletinRepository implements BulletinRepository {
     }
 
     /**
-     * Overrides the default bulletin processing strategy. When a custom
-     * bulletin strategy is employed, bulletins will not be persisted in this
-     * repository and will sent to the specified strategy instead.
+     * Overrides the default bulletin processing strategy. When a custom bulletin strategy is employed, bulletins will not be persisted in this repository and will sent to the specified strategy
+     * instead.
      *
      * @param strategy bulletin strategy
      */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
index 27eca37..f8ca2f0 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/fingerprint/FingerprintFactory.java
@@ -72,15 +72,10 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
 /**
- * Creates a fingerprint of a flow.xml. The order of elements or attributes in
- * the flow.xml does not influence the fingerprint generation.
+ * Creates a fingerprint of a flow.xml. The order of elements or attributes in the flow.xml does not influence the fingerprint generation.
  *
- * Only items in the flow.xml that influence the processing of data are
- * incorporated into the fingerprint. Examples of items involved in the
- * fingerprint are: processor IDs, processor relationships, and processor
- * properties. Examples of items not involved in the fingerprint are: items in
- * the processor "settings" or "comments" tabs, position information, flow
- * controller settings, and counters.
+ * Only items in the flow.xml that influence the processing of data are incorporated into the fingerprint. Examples of items involved in the fingerprint are: processor IDs, processor relationships,
+ * and processor properties. Examples of items not involved in the fingerprint are: items in the processor "settings" or "comments" tabs, position information, flow controller settings, and counters.
  *
  */
 public final class FingerprintFactory {
@@ -125,8 +120,7 @@ public final class FingerprintFactory {
     }
 
     /**
-     * Creates a fingerprint of a flow. The order of elements or attributes in
-     * the flow does not influence the fingerprint generation.
+     * Creates a fingerprint of a flow. The order of elements or attributes in the flow does not influence the fingerprint generation.
      *
      * @param flowBytes the flow represented as bytes
      * @param controller the controller
@@ -183,8 +177,7 @@ public final class FingerprintFactory {
     }
 
     /**
-     * Creates a fingerprint of a Collection of Templates The order of the
-     * templates does not influence the fingerprint generation.
+     * Creates a fingerprint of a Collection of Templates The order of the templates does not influence the fingerprint generation.
      *
      *
      * @param templates collection of templates

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
index e0181ea..c7baef4 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/groups/StandardProcessGroup.java
@@ -1805,14 +1805,11 @@ public final class StandardProcessGroup implements ProcessGroup {
     }
 
     /**
-     * Verifies that all ID's defined within the given snippet reference
-     * components within this ProcessGroup. If this is not the case, throws
-     * {@link IllegalStateException}.
+     * Verifies that all ID's defined within the given snippet reference components within this ProcessGroup. If this is not the case, throws {@link IllegalStateException}.
      *
      * @param snippet the snippet
      * @throws NullPointerException if the argument is null
-     * @throws IllegalStateException if the snippet contains an ID that
-     * references a component that is not part of this ProcessGroup
+     * @throws IllegalStateException if the snippet contains an ID that references a component that is not part of this ProcessGroup
      */
     private void verifyContents(final Snippet snippet) throws NullPointerException, IllegalStateException {
         requireNonNull(snippet);
@@ -1829,10 +1826,8 @@ public final class StandardProcessGroup implements ProcessGroup {
 
     /**
      * <p>
-     * Verifies that all ID's specified by the given set exist as keys in the
-     * given Map. If any of the ID's does not exist as a key in the map, will
-     * throw {@link IllegalStateException} indicating the ID that is invalid and
-     * specifying the Component Type.
+     * Verifies that all ID's specified by the given set exist as keys in the given Map. If any of the ID's does not exist as a key in the map, will throw {@link IllegalStateException} indicating the
+     * ID that is invalid and specifying the Component Type.
      * </p>
      *
      * <p>

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java
index 84f0dbc..72b129c 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycle.java
@@ -17,37 +17,30 @@
 package org.apache.nifi.lifecycle;
 
 /**
- * Represents a start/stop lifecyle for a component.  <code>start</code> should
- * only be called once per lifecyle unless otherwise documented by implementing
- * classes.
+ * Represents a start/stop lifecyle for a component.  <code>start</code> should only be called once per lifecyle unless otherwise documented by implementing classes.
  *
  * @author unattributed
  */
 public interface LifeCycle {
 
     /**
-     * Initiates the start state of the lifecyle. Should not throw an exception
-     * if the component is already running.
+     * Initiates the start state of the lifecyle. Should not throw an exception if the component is already running.
      *
      * @throws LifeCycleStartException if startup or initialization failed
      */
     void start() throws LifeCycleStartException;
 
     /**
-     * Initiates the stop state of the lifecycle. Should not throw an exception
-     * if the component is already stopped.
+     * Initiates the stop state of the lifecycle. Should not throw an exception if the component is already stopped.
      *
-     * @param force true if all efforts such as thread interruption should be
-     * attempted to stop the component; false if a graceful stopping should be
-     * employed
+     * @param force true if all efforts such as thread interruption should be attempted to stop the component; false if a graceful stopping should be employed
      *
      * @throws LifeCycleStopException if the shutdown failed
      */
     void stop(boolean force) throws LifeCycleStopException;
 
     /**
-     * @return true if the component is started, but not yet stopped; false
-     * otherwise
+     * @return true if the component is started, but not yet stopped; false otherwise
      */
     boolean isRunning();
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java
index 297a998..d3bf2bf 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleException.java
@@ -17,8 +17,7 @@
 package org.apache.nifi.lifecycle;
 
 /**
- * The base exception for issues encountered during the lifecycle of a class
- * implementing the <code>LifeCycle</code> interface.
+ * The base exception for issues encountered during the lifecycle of a class implementing the <code>LifeCycle</code> interface.
  *
  * @author unattributed
  */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java
index 8d2f726..725d840 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStartException.java
@@ -17,9 +17,7 @@
 package org.apache.nifi.lifecycle;
 
 /**
- * Represents the exceptional case when a problem is encountered during the
- * startup or initialization of a class implementing the <code>LifeCycle</code>
- * interface.
+ * Represents the exceptional case when a problem is encountered during the startup or initialization of a class implementing the <code>LifeCycle</code> interface.
  *
  * @author unattributed
  */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java
index be3779b..910e0a8 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/lifecycle/LifeCycleStopException.java
@@ -17,8 +17,7 @@
 package org.apache.nifi.lifecycle;
 
 /**
- * Represents the exceptional case when a problem is encountered during the
- * shutdown of a class implementing the <code>LifeCycle</code> interface.
+ * Represents the exceptional case when a problem is encountered during the shutdown of a class implementing the <code>LifeCycle</code> interface.
  *
  * @author unattributed
  */

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
index cc3dbea..92a0d84 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/persistence/FlowConfigurationDAO.java
@@ -32,24 +32,18 @@ import org.apache.nifi.controller.UninheritableFlowException;
 public interface FlowConfigurationDAO {
 
     /**
-     * Loads the given controller with the values from the given proposed flow.
-     * If loading the proposed flow configuration would cause the controller to
-     * orphan flow files, then an UninheritableFlowException is thrown.
+     * Loads the given controller with the values from the given proposed flow. If loading the proposed flow configuration would cause the controller to orphan flow files, then an
+     * UninheritableFlowException is thrown.
      *
-     * If the FlowSynchronizationException is thrown, then the controller may
-     * have changed some of its state and should no longer be used.
+     * If the FlowSynchronizationException is thrown, then the controller may have changed some of its state and should no longer be used.
      *
      * @param controller a controller
      * @param dataFlow the flow to load
      * @throws java.io.IOException
      *
-     * @throws FlowSerializationException if proposed flow is not a valid flow
-     * configuration file
-     * @throws UninheritableFlowException if the proposed flow cannot be loaded
-     * by the controller because in doing so would risk orphaning flow files
-     * @throws FlowSynchronizationException if updates to the controller failed.
-     * If this exception is thrown, then the controller should be considered
-     * unsafe to be used
+     * @throws FlowSerializationException if proposed flow is not a valid flow configuration file
+     * @throws UninheritableFlowException if the proposed flow cannot be loaded by the controller because in doing so would risk orphaning flow files
+     * @throws FlowSynchronizationException if updates to the controller failed. If this exception is thrown, then the controller should be considered unsafe to be used
      */
     void load(FlowController controller, DataFlow dataFlow)
             throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException;
@@ -76,8 +70,7 @@ public interface FlowConfigurationDAO {
      * @param flow to save
      * @throws NullPointerException if the given flow is null
      * @throws IOException If unable to persist state of given flow
-     * @throws IllegalStateException if FileFlowDAO not in proper state for
-     * saving
+     * @throws IllegalStateException if FileFlowDAO not in proper state for saving
      */
     void save(FlowController flow) throws IOException;
 
@@ -85,12 +78,10 @@ public interface FlowConfigurationDAO {
      * Saves all changes made to the given flow to the given File.
      *
      * @param flow to save
-     * @param outStream the OutputStream to which the FlowController will be
-     * written
+     * @param outStream the OutputStream to which the FlowController will be written
      * @throws NullPointerException if the given flow is null
      * @throws IOException If unable to persist state of given flow
-     * @throws IllegalStateException if FileFlowDAO not in proper state for
-     * saving
+     * @throws IllegalStateException if FileFlowDAO not in proper state for saving
      */
     void save(FlowController flow, OutputStream outStream) throws IOException;
 
@@ -98,12 +89,10 @@ public interface FlowConfigurationDAO {
      * Saves all changes made to the given flow to the given File.
      *
      * @param flow to save
-     * @param archive if true will also attempt to archive the flow
-     * configuration
+     * @param archive if true will also attempt to archive the flow configuration
      * @throws NullPointerException if the given flow is null
      * @throws IOException If unable to persist state of given flow
-     * @throws IllegalStateException if FileFlowDAO not in proper state for
-     * saving
+     * @throws IllegalStateException if FileFlowDAO not in proper state for saving
      */
     void save(FlowController flow, boolean archive) throws IOException;
 

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
index 3e1d1e6..d1bfacf 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardProcessContext.java
@@ -65,8 +65,7 @@ public class StandardProcessContext implements ProcessContext, ControllerService
 
     /**
      * <p>
-     * Returns the currently configured value for the property with the given
-     * name.
+     * Returns the currently configured value for the property with the given name.
      * </p>
      */
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardPropertyValue.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardPropertyValue.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardPropertyValue.java
index b320a61..acb86aa 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardPropertyValue.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/processor/StandardPropertyValue.java
@@ -39,15 +39,9 @@ public final class StandardPropertyValue implements PropertyValue {
     }
 
     /**
-     * Constructs a new StandardPropertyValue with the given value & service
-     * lookup and indicates whether or not the rawValue contains any NiFi
-     * Expressions. If it is unknown whether or not the value contains any NiFi
-     * Expressions, the
-     * {@link #StandardPropertyValue(String, ControllerServiceLookup)}
-     * constructor should be used or <code>true</code> should be passed.
-     * However, if it is known that the value contains no NiFi Expression, that
-     * information should be provided so that calls to
-     * {@link #evaluateAttributeExpressions()} are much more efficient
+     * Constructs a new StandardPropertyValue with the given value & service lookup and indicates whether or not the rawValue contains any NiFi Expressions. If it is unknown whether or not the value
+     * contains any NiFi Expressions, the {@link #StandardPropertyValue(String, ControllerServiceLookup)} constructor should be used or <code>true</code> should be passed. However, if it is known that
+     * the value contains no NiFi Expression, that information should be provided so that calls to {@link #evaluateAttributeExpressions()} are much more efficient
      *
      * @param rawValue value
      * @param serviceLookup lookup

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java
index bed5279..e80d383 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/RemoteNiFiUtils.java
@@ -120,12 +120,9 @@ public class RemoteNiFiUtils {
     }
 
     /**
-     * Returns the port on which the remote instance is listening for Flow File
-     * transfers, or <code>null</code> if the remote instance is not configured
-     * to use Site-to-Site transfers.
+     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance is not configured to use Site-to-Site transfers.
      *
-     * @param uri the base URI of the remote instance. This should include the
-     * path only to the nifi-api level, as well as the protocol, host, and port.
+     * @param uri the base URI of the remote instance. This should include the path only to the nifi-api level, as well as the protocol, host, and port.
      * @param timeoutMillis wait time in millis
      * @return port number
      * @throws IOException ex
@@ -158,9 +155,7 @@ public class RemoteNiFiUtils {
     }
 
     /**
-     * Returns the port on which the remote instance is listening for Flow File
-     * transfers, or <code>null</code> if the remote instance is not configured
-     * to use Site-to-Site transfers.
+     * Returns the port on which the remote instance is listening for Flow File transfers, or <code>null</code> if the remote instance is not configured to use Site-to-Site transfers.
      *
      * @param uri the full URI to fetch, including the path.
      * @return port

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
index f4b5975..d19b5c1 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/remote/StandardRemoteProcessGroup.java
@@ -72,9 +72,8 @@ import com.sun.jersey.api.client.ClientResponse.Status;
 import com.sun.jersey.api.client.UniformInterfaceException;
 
 /**
- * Represents the Root Process Group of a remote NiFi Instance. Holds
- * information about that remote instance, as well as {@link IncomingPort}s and
- * {@link OutgoingPort}s for communicating with the remote instance.
+ * Represents the Root Process Group of a remote NiFi Instance. Holds information about that remote instance, as well as {@link IncomingPort}s and {@link OutgoingPort}s for communicating with the
+ * remote instance.
  */
 public class StandardRemoteProcessGroup implements RemoteProcessGroup {
 
@@ -325,11 +324,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
 
     /**
-     * Changes the currently configured input ports to the ports described in
-     * the given set. If any port is currently configured that is not in the set
-     * given, that port will be shutdown and removed. If any port is currently
-     * not configured and is in the set given, that port will be instantiated
-     * and started.
+     * Changes the currently configured input ports to the ports described in the given set. If any port is currently configured that is not in the set given, that port will be shutdown and removed.
+     * If any port is currently not configured and is in the set given, that port will be instantiated and started.
      *
      * @param ports the new ports
      *
@@ -378,12 +374,10 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
 
     /**
-     * Returns a boolean indicating whether or not an Output Port exists with
-     * the given ID
+     * Returns a boolean indicating whether or not an Output Port exists with the given ID
      *
      * @param id identifier of port
-     * @return <code>true</code> if an Output Port exists with the given ID,
-     * <code>false</code> otherwise.
+     * @return <code>true</code> if an Output Port exists with the given ID, <code>false</code> otherwise.
      */
     public boolean containsOutputPort(final String id) {
         readLock.lock();
@@ -395,11 +389,8 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
 
     /**
-     * Changes the currently configured output ports to the ports described in
-     * the given set. If any port is currently configured that is not in the set
-     * given, that port will be shutdown and removed. If any port is currently
-     * not configured and is in the set given, that port will be instantiated
-     * and started.
+     * Changes the currently configured output ports to the ports described in the given set. If any port is currently configured that is not in the set given, that port will be shutdown and removed.
+     * If any port is currently not configured and is in the set given, that port will be instantiated and started.
      *
      * @param ports the new ports
      *
@@ -452,8 +443,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
      *
      *
      * @throws NullPointerException if the given output Port is null
-     * @throws IllegalStateException if the port does not belong to this remote
-     * process group
+     * @throws IllegalStateException if the port does not belong to this remote process group
      */
     @Override
     public void removeNonExistentPort(final RemoteGroupPort port) {
@@ -531,13 +521,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
 
     /**
-     * Adds an Output Port to this Remote Process Group that is described by
-     * this DTO.
+     * Adds an Output Port to this Remote Process Group that is described by this DTO.
      *
      * @param descriptor
      *
-     * @throws IllegalStateException if an Output Port already exists with the
-     * ID given by dto.getId()
+     * @throws IllegalStateException if an Output Port already exists with the ID given by dto.getId()
      */
     private void addOutputPort(final RemoteProcessGroupPortDescriptor descriptor) {
         writeLock.lock();
@@ -563,8 +551,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
 
     /**
      * @param portIdentifier the ID of the Port to send FlowFiles to
-     * @return {@link RemoteGroupPort} that can be used to send FlowFiles to the
-     * port whose ID is given on the remote instance
+     * @return {@link RemoteGroupPort} that can be used to send FlowFiles to the port whose ID is given on the remote instance
      */
     @Override
     public RemoteGroupPort getInputPort(final String portIdentifier) {
@@ -581,8 +568,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
 
     /**
-     * @return a set of {@link OutgoingPort}s used for transmitting FlowFiles to
-     * the remote instance
+     * @return a set of {@link OutgoingPort}s used for transmitting FlowFiles to the remote instance
      */
     @Override
     public Set<RemoteGroupPort> getInputPorts() {
@@ -597,13 +583,11 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
 
     /**
-     * Adds an InputPort to this ProcessGroup that is described by the given
-     * DTO.
+     * Adds an InputPort to this ProcessGroup that is described by the given DTO.
      *
      * @param descriptor port descriptor
      *
-     * @throws IllegalStateException if an Input Port already exists with the ID
-     * given by the ID of the DTO.
+     * @throws IllegalStateException if an Input Port already exists with the ID given by the ID of the DTO.
      */
     private void addInputPort(final RemoteProcessGroupPortDescriptor descriptor) {
         writeLock.lock();
@@ -643,8 +627,7 @@ public class StandardRemoteProcessGroup implements RemoteProcessGroup {
     }
 
     /**
-     * @return a set of {@link RemoteGroupPort}s used for receiving FlowFiles
-     * from the remote instance
+     * @return a set of {@link RemoteGroupPort}s used for receiving FlowFiles from the remote instance
      */
     @Override
     public Set<RemoteGroupPort> getOutputPorts() {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java
index 71cf969..e59a8d4 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/services/FlowService.java
@@ -29,16 +29,14 @@ import org.apache.nifi.controller.UninheritableFlowException;
 import org.apache.nifi.lifecycle.LifeCycle;
 
 /**
- * Defines the API level services available for carrying out file-based dataflow
- * operations.
+ * Defines the API level services available for carrying out file-based dataflow operations.
  *
  * @author unattributed
  */
 public interface FlowService extends LifeCycle {
 
     /**
-     * Immediately persists the state of the flow controller to the flow.xml
-     * file in a blocking call.
+     * Immediately persists the state of the flow controller to the flow.xml file in a blocking call.
      *
      * @throws NullPointerException if the given flow is null.
      * @throws IOException if any problem occurs creating/modifying file
@@ -46,19 +44,16 @@ public interface FlowService extends LifeCycle {
     void saveFlowChanges() throws IOException;
 
     /**
-     * Immediately persists the state of the flow controller to the given output
-     * stream in a blocking call.
+     * Immediately persists the state of the flow controller to the given output stream in a blocking call.
      *
-     * @param outStream the stream to which the FlowController is to be
-     * persisted
+     * @param outStream the stream to which the FlowController is to be persisted
      * @throws NullPointerException if the given flow is null.
      * @throws IOException if any problem occurs creating/modifying file
      */
     void saveFlowChanges(OutputStream outStream) throws IOException;
 
     /**
-     * Saves the given stream to the flow.xml file on disk. This method does not
-     * change the state of the flow controller.
+     * Saves the given stream to the flow.xml file on disk. This method does not change the state of the flow controller.
      *
      * @param is an input stream
      * @throws IOException if unable to save the flow
@@ -66,10 +61,8 @@ public interface FlowService extends LifeCycle {
     void overwriteFlow(InputStream is) throws IOException;
 
     /**
-     * Asynchronously saves the flow controller. The flow controller will be
-     * copied and immediately returned. If another call to save is made within
-     * that time the latest called state of the flow controller will be used. In
-     * database terms this technique is referred to as 'write-delay'.
+     * Asynchronously saves the flow controller. The flow controller will be copied and immediately returned. If another call to save is made within that time the latest called state of the flow
+     * controller will be used. In database terms this technique is referred to as 'write-delay'.
      *
      * @param delayUnit unit of delay
      * @param delay period of delay
@@ -77,47 +70,35 @@ public interface FlowService extends LifeCycle {
     void saveFlowChanges(TimeUnit delayUnit, long delay);
 
     /**
-     * Asynchronously saves the flow controller. The flow controller will be
-     * copied and immediately returned. If another call to save is made within
-     * that time the latest called state of the flow controller will be used. In
-     * database terms this technique is referred to as 'write-delay'.
+     * Asynchronously saves the flow controller. The flow controller will be copied and immediately returned. If another call to save is made within that time the latest called state of the flow
+     * controller will be used. In database terms this technique is referred to as 'write-delay'.
      *
      * @param delayUnit unit of delay
      * @param delay period of delay
-     * @param archive if true means the user wants the flow configuration to be
-     * archived as well
+     * @param archive if true means the user wants the flow configuration to be archived as well
      */
     void saveFlowChanges(TimeUnit delayUnit, long delay, boolean archive);
 
     /**
      * Stops the flow and underlying repository as determined by user
      *
-     * @param force if true the controller is not allowed to gracefully shut
-     * down.
+     * @param force if true the controller is not allowed to gracefully shut down.
      */
     @Override
     void stop(boolean force);
 
     /**
-     * Loads the flow controller with the given flow. Passing null means that
-     * the local flow on disk will used as the proposed flow. If loading the
-     * proposed flow configuration would cause the controller to orphan flow
-     * files, then an UninheritableFlowException is thrown.
+     * Loads the flow controller with the given flow. Passing null means that the local flow on disk will used as the proposed flow. If loading the proposed flow configuration would cause the
+     * controller to orphan flow files, then an UninheritableFlowException is thrown.
      *
-     * If the FlowSynchronizationException is thrown, then the controller may
-     * have changed some of its state and should no longer be used.
+     * If the FlowSynchronizationException is thrown, then the controller may have changed some of its state and should no longer be used.
      *
      * @param proposedFlow the flow to load
      *
-     * @throws IOException if flow configuration could not be retrieved from
-     * disk
-     * @throws FlowSerializationException if proposed flow is not a valid flow
-     * configuration file
-     * @throws UninheritableFlowException if the proposed flow cannot be loaded
-     * by the controller because in doing so would risk orphaning flow files
-     * @throws FlowSynchronizationException if updates to the controller failed.
-     * If this exception is thrown, then the controller should be considered
-     * unsafe to be used
+     * @throws IOException if flow configuration could not be retrieved from disk
+     * @throws FlowSerializationException if proposed flow is not a valid flow configuration file
+     * @throws UninheritableFlowException if the proposed flow cannot be loaded by the controller because in doing so would risk orphaning flow files
+     * @throws FlowSynchronizationException if updates to the controller failed. If this exception is thrown, then the controller should be considered unsafe to be used
      */
     void load(DataFlow proposedFlow)
             throws IOException, FlowSerializationException, FlowSynchronizationException, UninheritableFlowException;

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
index 665738b..6949cf9 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/FlowControllerFactoryBean.java
@@ -29,9 +29,7 @@ import org.springframework.context.ApplicationContext;
 import org.springframework.context.ApplicationContextAware;
 
 /**
- * Factory bean for creating a singleton FlowController instance. If the
- * application is configured to act as the cluster manager, then null is always
- * returned as the created instance.
+ * Factory bean for creating a singleton FlowController instance. If the application is configured to act as the cluster manager, then null is always returned as the created instance.
  */
 @SuppressWarnings("rawtypes")
 public class FlowControllerFactoryBean implements FactoryBean, ApplicationContextAware {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/StandardFlowServiceFactoryBean.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/StandardFlowServiceFactoryBean.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/StandardFlowServiceFactoryBean.java
index cfd855f..2aed5c2 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/StandardFlowServiceFactoryBean.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/spring/StandardFlowServiceFactoryBean.java
@@ -28,9 +28,7 @@ import org.springframework.context.ApplicationContext;
 import org.springframework.context.ApplicationContextAware;
 
 /**
- * Factory bean for creating a singleton FlowController instance. If the
- * application is configured to act as the cluster manager, then null is always
- * returned as the created instance.
+ * Factory bean for creating a singleton FlowController instance. If the application is configured to act as the cluster manager, then null is always returned as the created instance.
  */
 @SuppressWarnings("rawtypes")
 public class StandardFlowServiceFactoryBean implements FactoryBean, ApplicationContextAware {

http://git-wip-us.apache.org/repos/asf/incubator-nifi/blob/e811929f/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ComponentStatusReport.java
----------------------------------------------------------------------
diff --git a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ComponentStatusReport.java b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ComponentStatusReport.java
index 1296004..ca31467 100644
--- a/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ComponentStatusReport.java
+++ b/nifi/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/util/ComponentStatusReport.java
@@ -28,9 +28,7 @@ import org.apache.nifi.controller.status.ProcessorStatus;
 import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
 
 /**
- * ComponentStatusReport is a util class that can be used to "flatten" a
- * ProcessGroupStatus into a collection of Map's so that retrieval of a Status
- * for a particular component is very efficient
+ * ComponentStatusReport is a util class that can be used to "flatten" a ProcessGroupStatus into a collection of Map's so that retrieval of a Status for a particular component is very efficient
  */
 public class ComponentStatusReport {